From 193b572a0581b621069684e437daf93980e3fcbc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 17 Sep 2020 15:11:23 +0300 Subject: [PATCH 01/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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 c1abf5d13ca29b1fcf4bee0c8531491e931b0616 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Sun, 1 Nov 2020 02:56:41 +0300 Subject: [PATCH 29/94] *added adapters' boilerplate for Lzma buffers, *added submodule to gitmodules, *added cmake for xz --- .gitmodules | 6 + contrib/CMakeLists.txt | 1 + contrib/xz-cmake/CMakeLists.txt | 246 ++++++++++++++++++++++++++++++++ src/IO/CompressionMethod.cpp | 5 + src/IO/CompressionMethod.h | 6 +- src/IO/LzmaReadBuffer.cpp | 72 ++++++++++ src/IO/LzmaReadBuffer.h | 34 +++++ src/IO/LzmaWriteBuffer.cpp | 1 + src/IO/LzmaWriteBuffer.h | 10 ++ 9 files changed, 379 insertions(+), 2 deletions(-) create mode 100644 contrib/xz-cmake/CMakeLists.txt create mode 100644 src/IO/LzmaReadBuffer.cpp create mode 100644 src/IO/LzmaReadBuffer.h create mode 100644 src/IO/LzmaWriteBuffer.cpp create mode 100644 src/IO/LzmaWriteBuffer.h diff --git a/.gitmodules b/.gitmodules index 19f93ee8270..1545e92b54c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -186,3 +186,9 @@ [submodule "contrib/cyrus-sasl"] path = contrib/cyrus-sasl url = https://github.com/cyrusimap/cyrus-sasl +[submodule "contrib/xz-mirror"] + path = contrib/xz-mirror + url = https://github.com/xz-mirror/xz +[submodule "contrib/xz"] + path = contrib/xz + url = https://github.com/xz-mirror/xz diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 130e4b13c91..ec425c21239 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -32,6 +32,7 @@ add_subdirectory (murmurhash) add_subdirectory (replxx-cmake) add_subdirectory (ryu-cmake) add_subdirectory (unixodbc-cmake) +add_subdirectory (xz-cmake) add_subdirectory (poco-cmake) diff --git a/contrib/xz-cmake/CMakeLists.txt b/contrib/xz-cmake/CMakeLists.txt new file mode 100644 index 00000000000..d1295684d7f --- /dev/null +++ b/contrib/xz-cmake/CMakeLists.txt @@ -0,0 +1,246 @@ +############################################################################# +# +# Very limited CMake support for building some parts of XZ Utils +# +# For now, this is indented to be useful to build static or shared liblzma +# on Windows with MSVC (to avoid the need to maintain Visual Studio project +# files). Building liblzma on a few other platforms should work too but it +# is somewhat experimental and not as portable as using ./configure. +# +# On some platforms this builds also xz and xzdec, but these are +# highly experimental and meant for testing only: +# - No large file support on those 32-bit platforms that need it +# - No replacement getopt_long(), libc must have it +# - No sandboxing support +# - No translations +# - No xz symlinks are installed +# +# Other missing things: +# - No xzgrep or other scripts or their symlinks +# - No tests (no test failures either!) +# +# NOTE: Even if the code compiles without warnings, the end result may be +# different than via ./configure. Specifically, the list of #defines +# may be different (if so, probably this CMakeLists.txt got them wrong). +# +# This file provides the following installation components (if you only +# need liblzma, install only its components!): +# - liblzma_Runtime +# - liblzma_Development +# - xz (on some platforms only) +# - xzdec (on some platforms only) +# +# To find the target liblzma::liblzma from other packages, use the CONFIG +# option with find_package() to avoid a conflict with the FindLibLZMA module +# with case-insensitive file systems. For example, to require liblzma 5.2.5 +# or a newer compatible version: +# +# find_package(liblzma 5.2.5 REQUIRED CONFIG) +# target_link_libraries(my_application liblzma::liblzma) +# +############################################################################# +# +# Author: Lasse Collin +# +# This file has been put into the public domain. +# You can do whatever you want with this file. +# +############################################################################# + +# Define library directory, where sources and header files are located +SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/xz/src) + +# Read file with version +file(READ ${LIBRARY_DIR}/liblzma/api/lzma/version.h XZ_VERSION) +string(REGEX REPLACE +"^.*\n\ +#define LZMA_VERSION_MAJOR ([0-9]+)\n\ +#define LZMA_VERSION_MINOR ([0-9]+)\n\ +#define LZMA_VERSION_PATCH ([0-9]+)\n\ +.*$" + "\\1.\\2.\\3" XZ_VERSION "${XZ_VERSION}") + +# Parse version +MESSAGE(STATUS "LZMA VERSION ${XZ_VERSION}") + +# cd contrib/xz/src +# find . -name '*.c' | grep -vP 'deprecated|legacy|/xz/' | sort | sed 's/^\./ ${LIBRARY_DIR}/' +SET(Sources + ${LIBRARY_DIR}/common/tuklib_cpucores.c + ${LIBRARY_DIR}/common/tuklib_exit.c + ${LIBRARY_DIR}/common/tuklib_mbstr_fw.c + ${LIBRARY_DIR}/common/tuklib_mbstr_width.c + ${LIBRARY_DIR}/common/tuklib_open_stdxxx.c + ${LIBRARY_DIR}/common/tuklib_physmem.c + ${LIBRARY_DIR}/common/tuklib_progname.c + ${LIBRARY_DIR}/liblzma/check/check.c + ${LIBRARY_DIR}/liblzma/check/crc32_fast.c + ${LIBRARY_DIR}/liblzma/check/crc32_small.c + ${LIBRARY_DIR}/liblzma/check/crc32_table.c + ${LIBRARY_DIR}/liblzma/check/crc32_tablegen.c + ${LIBRARY_DIR}/liblzma/check/crc64_fast.c + ${LIBRARY_DIR}/liblzma/check/crc64_small.c + ${LIBRARY_DIR}/liblzma/check/crc64_table.c + ${LIBRARY_DIR}/liblzma/check/crc64_tablegen.c + ${LIBRARY_DIR}/liblzma/check/sha256.c + ${LIBRARY_DIR}/liblzma/common/alone_decoder.c + ${LIBRARY_DIR}/liblzma/common/alone_encoder.c + ${LIBRARY_DIR}/liblzma/common/auto_decoder.c + ${LIBRARY_DIR}/liblzma/common/block_buffer_decoder.c + ${LIBRARY_DIR}/liblzma/common/block_buffer_encoder.c + ${LIBRARY_DIR}/liblzma/common/block_decoder.c + ${LIBRARY_DIR}/liblzma/common/block_encoder.c + ${LIBRARY_DIR}/liblzma/common/block_header_decoder.c + ${LIBRARY_DIR}/liblzma/common/block_header_encoder.c + ${LIBRARY_DIR}/liblzma/common/block_util.c + ${LIBRARY_DIR}/liblzma/common/common.c + ${LIBRARY_DIR}/liblzma/common/easy_buffer_encoder.c + ${LIBRARY_DIR}/liblzma/common/easy_decoder_memusage.c + ${LIBRARY_DIR}/liblzma/common/easy_encoder.c + ${LIBRARY_DIR}/liblzma/common/easy_encoder_memusage.c + ${LIBRARY_DIR}/liblzma/common/easy_preset.c + ${LIBRARY_DIR}/liblzma/common/file_info.c + ${LIBRARY_DIR}/liblzma/common/filter_buffer_decoder.c + ${LIBRARY_DIR}/liblzma/common/filter_buffer_encoder.c + ${LIBRARY_DIR}/liblzma/common/filter_common.c + ${LIBRARY_DIR}/liblzma/common/filter_decoder.c + ${LIBRARY_DIR}/liblzma/common/filter_encoder.c + ${LIBRARY_DIR}/liblzma/common/filter_flags_decoder.c + ${LIBRARY_DIR}/liblzma/common/filter_flags_encoder.c + ${LIBRARY_DIR}/liblzma/common/hardware_cputhreads.c + ${LIBRARY_DIR}/liblzma/common/hardware_physmem.c + ${LIBRARY_DIR}/liblzma/common/index.c + ${LIBRARY_DIR}/liblzma/common/index_decoder.c + ${LIBRARY_DIR}/liblzma/common/index_encoder.c + ${LIBRARY_DIR}/liblzma/common/index_hash.c + ${LIBRARY_DIR}/liblzma/common/outqueue.c + ${LIBRARY_DIR}/liblzma/common/stream_buffer_decoder.c + ${LIBRARY_DIR}/liblzma/common/stream_buffer_encoder.c + ${LIBRARY_DIR}/liblzma/common/stream_decoder.c + ${LIBRARY_DIR}/liblzma/common/stream_encoder.c + ${LIBRARY_DIR}/liblzma/common/stream_encoder_mt.c + ${LIBRARY_DIR}/liblzma/common/stream_flags_common.c + ${LIBRARY_DIR}/liblzma/common/stream_flags_decoder.c + ${LIBRARY_DIR}/liblzma/common/stream_flags_encoder.c + ${LIBRARY_DIR}/liblzma/common/vli_decoder.c + ${LIBRARY_DIR}/liblzma/common/vli_encoder.c + ${LIBRARY_DIR}/liblzma/common/vli_size.c + ${LIBRARY_DIR}/liblzma/delta/delta_common.c + ${LIBRARY_DIR}/liblzma/delta/delta_decoder.c + ${LIBRARY_DIR}/liblzma/delta/delta_encoder.c + ${LIBRARY_DIR}/liblzma/lz/lz_decoder.c + ${LIBRARY_DIR}/liblzma/lz/lz_encoder.c + ${LIBRARY_DIR}/liblzma/lz/lz_encoder_mf.c + ${LIBRARY_DIR}/liblzma/lzma/fastpos_table.c + ${LIBRARY_DIR}/liblzma/lzma/fastpos_tablegen.c + ${LIBRARY_DIR}/liblzma/lzma/lzma2_decoder.c + ${LIBRARY_DIR}/liblzma/lzma/lzma2_encoder.c + ${LIBRARY_DIR}/liblzma/lzma/lzma_decoder.c + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder.c + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_optimum_fast.c + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_optimum_normal.c + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_presets.c + ${LIBRARY_DIR}/liblzma/rangecoder/price_table.c + ${LIBRARY_DIR}/liblzma/rangecoder/price_tablegen.c + ${LIBRARY_DIR}/liblzma/simple/arm.c + ${LIBRARY_DIR}/liblzma/simple/armthumb.c + ${LIBRARY_DIR}/liblzma/simple/ia64.c + ${LIBRARY_DIR}/liblzma/simple/powerpc.c + ${LIBRARY_DIR}/liblzma/simple/simple_coder.c + ${LIBRARY_DIR}/liblzma/simple/simple_decoder.c + ${LIBRARY_DIR}/liblzma/simple/simple_encoder.c + ${LIBRARY_DIR}/liblzma/simple/sparc.c + ${LIBRARY_DIR}/liblzma/simple/x86.c + ${LIBRARY_DIR}/lzmainfo/lzmainfo.c +) + +# cd contrib/xz/src +# find . -name '*.h' | grep -vP 'deprecated|legacy|/xz/' | sort | sed 's/^\./ ${LIBRARY_DIR}/' +SET(Headers + ${LIBRARY_DIR}/common/mythread.h + ${LIBRARY_DIR}/common/sysdefs.h + ${LIBRARY_DIR}/common/tuklib_common.h + ${LIBRARY_DIR}/common/tuklib_config.h + ${LIBRARY_DIR}/common/tuklib_cpucores.h + ${LIBRARY_DIR}/common/tuklib_exit.h + ${LIBRARY_DIR}/common/tuklib_gettext.h + ${LIBRARY_DIR}/common/tuklib_integer.h + ${LIBRARY_DIR}/common/tuklib_mbstr.h + ${LIBRARY_DIR}/common/tuklib_open_stdxxx.h + ${LIBRARY_DIR}/common/tuklib_physmem.h + ${LIBRARY_DIR}/common/tuklib_progname.h + ${LIBRARY_DIR}/liblzma/api/lzma/base.h + ${LIBRARY_DIR}/liblzma/api/lzma/bcj.h + ${LIBRARY_DIR}/liblzma/api/lzma/block.h + ${LIBRARY_DIR}/liblzma/api/lzma/check.h + ${LIBRARY_DIR}/liblzma/api/lzma/container.h + ${LIBRARY_DIR}/liblzma/api/lzma/delta.h + ${LIBRARY_DIR}/liblzma/api/lzma/filter.h + ${LIBRARY_DIR}/liblzma/api/lzma.h + ${LIBRARY_DIR}/liblzma/api/lzma/hardware.h + ${LIBRARY_DIR}/liblzma/api/lzma/index.h + ${LIBRARY_DIR}/liblzma/api/lzma/index_hash.h + ${LIBRARY_DIR}/liblzma/api/lzma/lzma12.h + ${LIBRARY_DIR}/liblzma/api/lzma/stream_flags.h + ${LIBRARY_DIR}/liblzma/api/lzma/version.h + ${LIBRARY_DIR}/liblzma/api/lzma/vli.h + ${LIBRARY_DIR}/liblzma/check/check.h + ${LIBRARY_DIR}/liblzma/check/crc32_table_be.h + ${LIBRARY_DIR}/liblzma/check/crc32_table_le.h + ${LIBRARY_DIR}/liblzma/check/crc64_table_be.h + ${LIBRARY_DIR}/liblzma/check/crc64_table_le.h + ${LIBRARY_DIR}/liblzma/check/crc_macros.h + ${LIBRARY_DIR}/liblzma/common/alone_decoder.h + ${LIBRARY_DIR}/liblzma/common/block_buffer_encoder.h + ${LIBRARY_DIR}/liblzma/common/block_decoder.h + ${LIBRARY_DIR}/liblzma/common/block_encoder.h + ${LIBRARY_DIR}/liblzma/common/common.h + ${LIBRARY_DIR}/liblzma/common/easy_preset.h + ${LIBRARY_DIR}/liblzma/common/filter_common.h + ${LIBRARY_DIR}/liblzma/common/filter_decoder.h + ${LIBRARY_DIR}/liblzma/common/filter_encoder.h + ${LIBRARY_DIR}/liblzma/common/index_decoder.h + ${LIBRARY_DIR}/liblzma/common/index_encoder.h + ${LIBRARY_DIR}/liblzma/common/index.h + ${LIBRARY_DIR}/liblzma/common/memcmplen.h + ${LIBRARY_DIR}/liblzma/common/outqueue.h + ${LIBRARY_DIR}/liblzma/common/stream_decoder.h + ${LIBRARY_DIR}/liblzma/common/stream_flags_common.h + ${LIBRARY_DIR}/liblzma/delta/delta_common.h + ${LIBRARY_DIR}/liblzma/delta/delta_decoder.h + ${LIBRARY_DIR}/liblzma/delta/delta_encoder.h + ${LIBRARY_DIR}/liblzma/delta/delta_private.h + ${LIBRARY_DIR}/liblzma/lz/lz_decoder.h + ${LIBRARY_DIR}/liblzma/lz/lz_encoder.h + ${LIBRARY_DIR}/liblzma/lz/lz_encoder_hash.h + ${LIBRARY_DIR}/liblzma/lz/lz_encoder_hash_table.h + ${LIBRARY_DIR}/liblzma/lzma/fastpos.h + ${LIBRARY_DIR}/liblzma/lzma/lzma2_decoder.h + ${LIBRARY_DIR}/liblzma/lzma/lzma2_encoder.h + ${LIBRARY_DIR}/liblzma/lzma/lzma_common.h + ${LIBRARY_DIR}/liblzma/lzma/lzma_decoder.h + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder.h + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_private.h + ${LIBRARY_DIR}/liblzma/rangecoder/price.h + ${LIBRARY_DIR}/liblzma/rangecoder/range_common.h + ${LIBRARY_DIR}/liblzma/rangecoder/range_decoder.h + ${LIBRARY_DIR}/liblzma/rangecoder/range_encoder.h + ${LIBRARY_DIR}/liblzma/simple/simple_coder.h + ${LIBRARY_DIR}/liblzma/simple/simple_decoder.h + ${LIBRARY_DIR}/liblzma/simple/simple_encoder.h + ${LIBRARY_DIR}/liblzma/simple/simple_private.h +) + +ADD_LIBRARY(liblzma ${Sources} ${Headers}) + +target_include_directories(liblzma PUBLIC + ${LIBRARY_DIR}/liblzma/api + ${LIBRARY_DIR}/liblzma/common + ${LIBRARY_DIR}/liblzma/check + ${LIBRARY_DIR}/liblzma/lz + ${LIBRARY_DIR}/liblzma/rangecoder + ${LIBRARY_DIR}/liblzma/lzma + ${LIBRARY_DIR}/liblzma/delta + ${LIBRARY_DIR}/liblzma/simple + ${LIBRARY_DIR}/common +) \ No newline at end of file diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index a0a5e19f4fa..0bf390d92ca 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #if !defined(ARCADIA_BUILD) # include @@ -28,6 +30,7 @@ std::string toContentEncodingName(CompressionMethod method) case CompressionMethod::Gzip: return "gzip"; case CompressionMethod::Zlib: return "deflate"; case CompressionMethod::Brotli: return "br"; + case CompressionMethod::Xz: return "xz"; case CompressionMethod::None: return ""; } __builtin_unreachable(); @@ -73,6 +76,8 @@ std::unique_ptr wrapReadBufferWithCompressionMethod( if (method == CompressionMethod::Brotli) return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); #endif + if (method == CompressionMethod::Xz) + return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); if (method == CompressionMethod::None) return nested; diff --git a/src/IO/CompressionMethod.h b/src/IO/CompressionMethod.h index 64c2ba3341f..5b0d4330011 100644 --- a/src/IO/CompressionMethod.h +++ b/src/IO/CompressionMethod.h @@ -1,14 +1,13 @@ #pragma once -#include #include +#include #include namespace DB { - class ReadBuffer; class WriteBuffer; @@ -26,6 +25,9 @@ enum class CompressionMethod /// DEFLATE compression with zlib header and Adler32 checksum. /// This option corresponds to HTTP Content-Encoding: deflate. Zlib, + /// LZMA2-based content compression + /// This option corresponds to HTTP Content-Encoding: xz + Xz, Brotli }; diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp new file mode 100644 index 00000000000..a241067f8bc --- /dev/null +++ b/src/IO/LzmaReadBuffer.cpp @@ -0,0 +1,72 @@ +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LZMA_STREAM_DECODER_FAILED; +} +LzmaReadBuffer::LzmaReadBuffer( + std::unique_ptr in_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) + : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)) +{ + lstr.allocator = nullptr; + lstr.next_in = nullptr; + lstr.avail_in = 0; + lstr.next_out = nullptr; + lstr.avail_out = 0; + + // 500 mb + uint64_t memlimit = 500 << 30; + + lstr = LZMA_STREAM_INIT; + lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED | LZMA_IGNORE_CHECK); + // lzma does not provide api for converting error code to string unlike zlib + if (ret != LZMA_OK) + throw Exception( + std::string("lzma_stream_decoder failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_DECODER_FAILED); +} + +LzmaReadBuffer::~LzmaReadBuffer() +{ + lzma_end(&lstr); +} + +bool LzmaReadBuffer::nextImpl() +{ + if (eof) + { + return false; + } + + if (!lstr.avail_in) + { + in->nextIfAtEnd(); + lstr.next_in = reinterpret_cast(in->position()); + lstr.avail_in = in->buffer().end() - in->position(); + } + lstr.next_out = reinterpret_cast(internal_buffer.begin()); + lstr.avail_out = internal_buffer.size(); + + lzma_ret ret = lzma_code(&lstr, LZMA_FINISH); + + in->position() = in->buffer().end() - lstr.avail_in; + + if (ret == LZMA_STREAM_END) + { + if (in->eof()) + { + eof = true; + return working_buffer.size() != 0; + } + } + + if (ret != LZMA_OK) + throw Exception( + std::string("lzma_stream_decoder failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_DECODER_FAILED); + + return true +} +} \ No newline at end of file diff --git a/src/IO/LzmaReadBuffer.h b/src/IO/LzmaReadBuffer.h new file mode 100644 index 00000000000..3ece8a46b7d --- /dev/null +++ b/src/IO/LzmaReadBuffer.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ +} + +class LzmaReadBuffer : public BufferWithOwnMemory +{ +public: + LzmaReadBuffer( + std::unique_ptr in_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + + ~LzmaReadBuffer() override; + +private: + bool nextImpl() override; + + std::unique_ptr in; + lzma_stream lstr; + bool eof; +}; + +} \ No newline at end of file diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp new file mode 100644 index 00000000000..b9d8a3e1f2a --- /dev/null +++ b/src/IO/LzmaWriteBuffer.cpp @@ -0,0 +1 @@ +#include \ No newline at end of file diff --git a/src/IO/LzmaWriteBuffer.h b/src/IO/LzmaWriteBuffer.h new file mode 100644 index 00000000000..6824b88b21b --- /dev/null +++ b/src/IO/LzmaWriteBuffer.h @@ -0,0 +1,10 @@ +#pragma once + + +#include + +namespace DB { + + + +} \ No newline at end of file From 5982f9f7e56411f186f1ea4b9c1cf0f1480df1e4 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Sun, 1 Nov 2020 03:00:36 +0300 Subject: [PATCH 30/94] removed extra record from cmake --- .gitmodules | 3 --- 1 file changed, 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index 1545e92b54c..9ad672aa17f 100644 --- a/.gitmodules +++ b/.gitmodules @@ -186,9 +186,6 @@ [submodule "contrib/cyrus-sasl"] path = contrib/cyrus-sasl url = https://github.com/cyrusimap/cyrus-sasl -[submodule "contrib/xz-mirror"] - path = contrib/xz-mirror - url = https://github.com/xz-mirror/xz [submodule "contrib/xz"] path = contrib/xz url = https://github.com/xz-mirror/xz From 731e274cd0a464adc27ead4ceca974257dc1711f Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Sun, 1 Nov 2020 03:14:40 +0300 Subject: [PATCH 31/94] changed moment of stream object initialization --- src/IO/LzmaReadBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index a241067f8bc..1150a8f48e6 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -10,6 +10,7 @@ LzmaReadBuffer::LzmaReadBuffer( std::unique_ptr in_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)) { + lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; lstr.next_in = nullptr; lstr.avail_in = 0; @@ -19,7 +20,6 @@ LzmaReadBuffer::LzmaReadBuffer( // 500 mb uint64_t memlimit = 500 << 30; - lstr = LZMA_STREAM_INIT; lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED | LZMA_IGNORE_CHECK); // lzma does not provide api for converting error code to string unlike zlib if (ret != LZMA_OK) From be2b002ff49e74fb6d2931e3af511c6b0c818db1 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Sun, 1 Nov 2020 21:40:05 +0300 Subject: [PATCH 32/94] fixed cmake for building, added test for compression method, added basic code for encoding --- contrib/CMakeLists.txt | 2 +- contrib/xz-cmake/CMakeLists.txt | 246 -------------------------------- src/CMakeLists.txt | 7 + src/Common/ErrorCodes.cpp | 3 + src/IO/LzmaReadBuffer.cpp | 7 +- src/IO/LzmaWriteBuffer.cpp | 119 ++++++++++++++- src/IO/LzmaWriteBuffer.h | 25 +++- src/IO/tests/CMakeLists.txt | 3 + src/IO/tests/lzma_buffers.cpp | 66 +++++++++ 9 files changed, 227 insertions(+), 251 deletions(-) delete mode 100644 contrib/xz-cmake/CMakeLists.txt create mode 100644 src/IO/tests/lzma_buffers.cpp diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index ec425c21239..695f0acf6db 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -32,7 +32,7 @@ add_subdirectory (murmurhash) add_subdirectory (replxx-cmake) add_subdirectory (ryu-cmake) add_subdirectory (unixodbc-cmake) -add_subdirectory (xz-cmake) +add_subdirectory (xz) add_subdirectory (poco-cmake) diff --git a/contrib/xz-cmake/CMakeLists.txt b/contrib/xz-cmake/CMakeLists.txt deleted file mode 100644 index d1295684d7f..00000000000 --- a/contrib/xz-cmake/CMakeLists.txt +++ /dev/null @@ -1,246 +0,0 @@ -############################################################################# -# -# Very limited CMake support for building some parts of XZ Utils -# -# For now, this is indented to be useful to build static or shared liblzma -# on Windows with MSVC (to avoid the need to maintain Visual Studio project -# files). Building liblzma on a few other platforms should work too but it -# is somewhat experimental and not as portable as using ./configure. -# -# On some platforms this builds also xz and xzdec, but these are -# highly experimental and meant for testing only: -# - No large file support on those 32-bit platforms that need it -# - No replacement getopt_long(), libc must have it -# - No sandboxing support -# - No translations -# - No xz symlinks are installed -# -# Other missing things: -# - No xzgrep or other scripts or their symlinks -# - No tests (no test failures either!) -# -# NOTE: Even if the code compiles without warnings, the end result may be -# different than via ./configure. Specifically, the list of #defines -# may be different (if so, probably this CMakeLists.txt got them wrong). -# -# This file provides the following installation components (if you only -# need liblzma, install only its components!): -# - liblzma_Runtime -# - liblzma_Development -# - xz (on some platforms only) -# - xzdec (on some platforms only) -# -# To find the target liblzma::liblzma from other packages, use the CONFIG -# option with find_package() to avoid a conflict with the FindLibLZMA module -# with case-insensitive file systems. For example, to require liblzma 5.2.5 -# or a newer compatible version: -# -# find_package(liblzma 5.2.5 REQUIRED CONFIG) -# target_link_libraries(my_application liblzma::liblzma) -# -############################################################################# -# -# Author: Lasse Collin -# -# This file has been put into the public domain. -# You can do whatever you want with this file. -# -############################################################################# - -# Define library directory, where sources and header files are located -SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/xz/src) - -# Read file with version -file(READ ${LIBRARY_DIR}/liblzma/api/lzma/version.h XZ_VERSION) -string(REGEX REPLACE -"^.*\n\ -#define LZMA_VERSION_MAJOR ([0-9]+)\n\ -#define LZMA_VERSION_MINOR ([0-9]+)\n\ -#define LZMA_VERSION_PATCH ([0-9]+)\n\ -.*$" - "\\1.\\2.\\3" XZ_VERSION "${XZ_VERSION}") - -# Parse version -MESSAGE(STATUS "LZMA VERSION ${XZ_VERSION}") - -# cd contrib/xz/src -# find . -name '*.c' | grep -vP 'deprecated|legacy|/xz/' | sort | sed 's/^\./ ${LIBRARY_DIR}/' -SET(Sources - ${LIBRARY_DIR}/common/tuklib_cpucores.c - ${LIBRARY_DIR}/common/tuklib_exit.c - ${LIBRARY_DIR}/common/tuklib_mbstr_fw.c - ${LIBRARY_DIR}/common/tuklib_mbstr_width.c - ${LIBRARY_DIR}/common/tuklib_open_stdxxx.c - ${LIBRARY_DIR}/common/tuklib_physmem.c - ${LIBRARY_DIR}/common/tuklib_progname.c - ${LIBRARY_DIR}/liblzma/check/check.c - ${LIBRARY_DIR}/liblzma/check/crc32_fast.c - ${LIBRARY_DIR}/liblzma/check/crc32_small.c - ${LIBRARY_DIR}/liblzma/check/crc32_table.c - ${LIBRARY_DIR}/liblzma/check/crc32_tablegen.c - ${LIBRARY_DIR}/liblzma/check/crc64_fast.c - ${LIBRARY_DIR}/liblzma/check/crc64_small.c - ${LIBRARY_DIR}/liblzma/check/crc64_table.c - ${LIBRARY_DIR}/liblzma/check/crc64_tablegen.c - ${LIBRARY_DIR}/liblzma/check/sha256.c - ${LIBRARY_DIR}/liblzma/common/alone_decoder.c - ${LIBRARY_DIR}/liblzma/common/alone_encoder.c - ${LIBRARY_DIR}/liblzma/common/auto_decoder.c - ${LIBRARY_DIR}/liblzma/common/block_buffer_decoder.c - ${LIBRARY_DIR}/liblzma/common/block_buffer_encoder.c - ${LIBRARY_DIR}/liblzma/common/block_decoder.c - ${LIBRARY_DIR}/liblzma/common/block_encoder.c - ${LIBRARY_DIR}/liblzma/common/block_header_decoder.c - ${LIBRARY_DIR}/liblzma/common/block_header_encoder.c - ${LIBRARY_DIR}/liblzma/common/block_util.c - ${LIBRARY_DIR}/liblzma/common/common.c - ${LIBRARY_DIR}/liblzma/common/easy_buffer_encoder.c - ${LIBRARY_DIR}/liblzma/common/easy_decoder_memusage.c - ${LIBRARY_DIR}/liblzma/common/easy_encoder.c - ${LIBRARY_DIR}/liblzma/common/easy_encoder_memusage.c - ${LIBRARY_DIR}/liblzma/common/easy_preset.c - ${LIBRARY_DIR}/liblzma/common/file_info.c - ${LIBRARY_DIR}/liblzma/common/filter_buffer_decoder.c - ${LIBRARY_DIR}/liblzma/common/filter_buffer_encoder.c - ${LIBRARY_DIR}/liblzma/common/filter_common.c - ${LIBRARY_DIR}/liblzma/common/filter_decoder.c - ${LIBRARY_DIR}/liblzma/common/filter_encoder.c - ${LIBRARY_DIR}/liblzma/common/filter_flags_decoder.c - ${LIBRARY_DIR}/liblzma/common/filter_flags_encoder.c - ${LIBRARY_DIR}/liblzma/common/hardware_cputhreads.c - ${LIBRARY_DIR}/liblzma/common/hardware_physmem.c - ${LIBRARY_DIR}/liblzma/common/index.c - ${LIBRARY_DIR}/liblzma/common/index_decoder.c - ${LIBRARY_DIR}/liblzma/common/index_encoder.c - ${LIBRARY_DIR}/liblzma/common/index_hash.c - ${LIBRARY_DIR}/liblzma/common/outqueue.c - ${LIBRARY_DIR}/liblzma/common/stream_buffer_decoder.c - ${LIBRARY_DIR}/liblzma/common/stream_buffer_encoder.c - ${LIBRARY_DIR}/liblzma/common/stream_decoder.c - ${LIBRARY_DIR}/liblzma/common/stream_encoder.c - ${LIBRARY_DIR}/liblzma/common/stream_encoder_mt.c - ${LIBRARY_DIR}/liblzma/common/stream_flags_common.c - ${LIBRARY_DIR}/liblzma/common/stream_flags_decoder.c - ${LIBRARY_DIR}/liblzma/common/stream_flags_encoder.c - ${LIBRARY_DIR}/liblzma/common/vli_decoder.c - ${LIBRARY_DIR}/liblzma/common/vli_encoder.c - ${LIBRARY_DIR}/liblzma/common/vli_size.c - ${LIBRARY_DIR}/liblzma/delta/delta_common.c - ${LIBRARY_DIR}/liblzma/delta/delta_decoder.c - ${LIBRARY_DIR}/liblzma/delta/delta_encoder.c - ${LIBRARY_DIR}/liblzma/lz/lz_decoder.c - ${LIBRARY_DIR}/liblzma/lz/lz_encoder.c - ${LIBRARY_DIR}/liblzma/lz/lz_encoder_mf.c - ${LIBRARY_DIR}/liblzma/lzma/fastpos_table.c - ${LIBRARY_DIR}/liblzma/lzma/fastpos_tablegen.c - ${LIBRARY_DIR}/liblzma/lzma/lzma2_decoder.c - ${LIBRARY_DIR}/liblzma/lzma/lzma2_encoder.c - ${LIBRARY_DIR}/liblzma/lzma/lzma_decoder.c - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder.c - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_optimum_fast.c - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_optimum_normal.c - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_presets.c - ${LIBRARY_DIR}/liblzma/rangecoder/price_table.c - ${LIBRARY_DIR}/liblzma/rangecoder/price_tablegen.c - ${LIBRARY_DIR}/liblzma/simple/arm.c - ${LIBRARY_DIR}/liblzma/simple/armthumb.c - ${LIBRARY_DIR}/liblzma/simple/ia64.c - ${LIBRARY_DIR}/liblzma/simple/powerpc.c - ${LIBRARY_DIR}/liblzma/simple/simple_coder.c - ${LIBRARY_DIR}/liblzma/simple/simple_decoder.c - ${LIBRARY_DIR}/liblzma/simple/simple_encoder.c - ${LIBRARY_DIR}/liblzma/simple/sparc.c - ${LIBRARY_DIR}/liblzma/simple/x86.c - ${LIBRARY_DIR}/lzmainfo/lzmainfo.c -) - -# cd contrib/xz/src -# find . -name '*.h' | grep -vP 'deprecated|legacy|/xz/' | sort | sed 's/^\./ ${LIBRARY_DIR}/' -SET(Headers - ${LIBRARY_DIR}/common/mythread.h - ${LIBRARY_DIR}/common/sysdefs.h - ${LIBRARY_DIR}/common/tuklib_common.h - ${LIBRARY_DIR}/common/tuklib_config.h - ${LIBRARY_DIR}/common/tuklib_cpucores.h - ${LIBRARY_DIR}/common/tuklib_exit.h - ${LIBRARY_DIR}/common/tuklib_gettext.h - ${LIBRARY_DIR}/common/tuklib_integer.h - ${LIBRARY_DIR}/common/tuklib_mbstr.h - ${LIBRARY_DIR}/common/tuklib_open_stdxxx.h - ${LIBRARY_DIR}/common/tuklib_physmem.h - ${LIBRARY_DIR}/common/tuklib_progname.h - ${LIBRARY_DIR}/liblzma/api/lzma/base.h - ${LIBRARY_DIR}/liblzma/api/lzma/bcj.h - ${LIBRARY_DIR}/liblzma/api/lzma/block.h - ${LIBRARY_DIR}/liblzma/api/lzma/check.h - ${LIBRARY_DIR}/liblzma/api/lzma/container.h - ${LIBRARY_DIR}/liblzma/api/lzma/delta.h - ${LIBRARY_DIR}/liblzma/api/lzma/filter.h - ${LIBRARY_DIR}/liblzma/api/lzma.h - ${LIBRARY_DIR}/liblzma/api/lzma/hardware.h - ${LIBRARY_DIR}/liblzma/api/lzma/index.h - ${LIBRARY_DIR}/liblzma/api/lzma/index_hash.h - ${LIBRARY_DIR}/liblzma/api/lzma/lzma12.h - ${LIBRARY_DIR}/liblzma/api/lzma/stream_flags.h - ${LIBRARY_DIR}/liblzma/api/lzma/version.h - ${LIBRARY_DIR}/liblzma/api/lzma/vli.h - ${LIBRARY_DIR}/liblzma/check/check.h - ${LIBRARY_DIR}/liblzma/check/crc32_table_be.h - ${LIBRARY_DIR}/liblzma/check/crc32_table_le.h - ${LIBRARY_DIR}/liblzma/check/crc64_table_be.h - ${LIBRARY_DIR}/liblzma/check/crc64_table_le.h - ${LIBRARY_DIR}/liblzma/check/crc_macros.h - ${LIBRARY_DIR}/liblzma/common/alone_decoder.h - ${LIBRARY_DIR}/liblzma/common/block_buffer_encoder.h - ${LIBRARY_DIR}/liblzma/common/block_decoder.h - ${LIBRARY_DIR}/liblzma/common/block_encoder.h - ${LIBRARY_DIR}/liblzma/common/common.h - ${LIBRARY_DIR}/liblzma/common/easy_preset.h - ${LIBRARY_DIR}/liblzma/common/filter_common.h - ${LIBRARY_DIR}/liblzma/common/filter_decoder.h - ${LIBRARY_DIR}/liblzma/common/filter_encoder.h - ${LIBRARY_DIR}/liblzma/common/index_decoder.h - ${LIBRARY_DIR}/liblzma/common/index_encoder.h - ${LIBRARY_DIR}/liblzma/common/index.h - ${LIBRARY_DIR}/liblzma/common/memcmplen.h - ${LIBRARY_DIR}/liblzma/common/outqueue.h - ${LIBRARY_DIR}/liblzma/common/stream_decoder.h - ${LIBRARY_DIR}/liblzma/common/stream_flags_common.h - ${LIBRARY_DIR}/liblzma/delta/delta_common.h - ${LIBRARY_DIR}/liblzma/delta/delta_decoder.h - ${LIBRARY_DIR}/liblzma/delta/delta_encoder.h - ${LIBRARY_DIR}/liblzma/delta/delta_private.h - ${LIBRARY_DIR}/liblzma/lz/lz_decoder.h - ${LIBRARY_DIR}/liblzma/lz/lz_encoder.h - ${LIBRARY_DIR}/liblzma/lz/lz_encoder_hash.h - ${LIBRARY_DIR}/liblzma/lz/lz_encoder_hash_table.h - ${LIBRARY_DIR}/liblzma/lzma/fastpos.h - ${LIBRARY_DIR}/liblzma/lzma/lzma2_decoder.h - ${LIBRARY_DIR}/liblzma/lzma/lzma2_encoder.h - ${LIBRARY_DIR}/liblzma/lzma/lzma_common.h - ${LIBRARY_DIR}/liblzma/lzma/lzma_decoder.h - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder.h - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_private.h - ${LIBRARY_DIR}/liblzma/rangecoder/price.h - ${LIBRARY_DIR}/liblzma/rangecoder/range_common.h - ${LIBRARY_DIR}/liblzma/rangecoder/range_decoder.h - ${LIBRARY_DIR}/liblzma/rangecoder/range_encoder.h - ${LIBRARY_DIR}/liblzma/simple/simple_coder.h - ${LIBRARY_DIR}/liblzma/simple/simple_decoder.h - ${LIBRARY_DIR}/liblzma/simple/simple_encoder.h - ${LIBRARY_DIR}/liblzma/simple/simple_private.h -) - -ADD_LIBRARY(liblzma ${Sources} ${Headers}) - -target_include_directories(liblzma PUBLIC - ${LIBRARY_DIR}/liblzma/api - ${LIBRARY_DIR}/liblzma/common - ${LIBRARY_DIR}/liblzma/check - ${LIBRARY_DIR}/liblzma/lz - ${LIBRARY_DIR}/liblzma/rangecoder - ${LIBRARY_DIR}/liblzma/lzma - ${LIBRARY_DIR}/liblzma/delta - ${LIBRARY_DIR}/liblzma/simple - ${LIBRARY_DIR}/common -) \ No newline at end of file diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 843dd8c2615..7d76a910b17 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -318,6 +318,13 @@ if (ZSTD_LIBRARY) endif () endif() +set (LZMA_LIBRARY liblzma) +set (LZMA_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/xz/src/liblzma/api) +if (LZMA_LIBRARY) + target_link_libraries (clickhouse_common_io PUBLIC ${LZMA_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${LZMA_INCLUDE_DIR}) +endif() + if (USE_ICU) dbms_target_link_libraries (PRIVATE ${ICU_LIBRARIES}) dbms_target_include_directories (SYSTEM PRIVATE ${ICU_INCLUDE_DIRS}) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b75f885e559..1bb96fdb7a4 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -512,6 +512,9 @@ namespace ErrorCodes extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND = 2001; extern const int ILLEGAL_PROJECTION_MANIPULATOR = 2002; extern const int UNRECOGNIZED_ARGUMENTS = 2003; + + extern const int LZMA_STREAM_ENCODER_FAILED = 2004; + extern const int LZMA_STREAM_DECODER_FAILED = 2005; } } diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 1150a8f48e6..18cca25b4d5 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -7,7 +7,10 @@ namespace ErrorCodes extern const int LZMA_STREAM_DECODER_FAILED; } LzmaReadBuffer::LzmaReadBuffer( - std::unique_ptr in_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) + std::unique_ptr in_, + size_t buf_size, + char * existing_memory, + size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)) { lstr = LZMA_STREAM_INIT; @@ -67,6 +70,6 @@ bool LzmaReadBuffer::nextImpl() std::string("lzma_stream_decoder failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_DECODER_FAILED); - return true + return true; } } \ No newline at end of file diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp index b9d8a3e1f2a..6ca867d2af7 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LzmaWriteBuffer.cpp @@ -1 +1,118 @@ -#include \ No newline at end of file +#include + + +namespace DB { +namespace ErrorCodes +{ + extern const int LZMA_STREAM_ENCODER_FAILED; +} + +LzmaWriteBuffer::LzmaWriteBuffer( + std::unique_ptr out_, + int compression_level, + size_t buf_size, + char * existing_memory, + size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment) + , out(std::move(out_)) +{ + lstr = LZMA_STREAM_INIT; + lstr.allocator = nullptr; + lstr.next_in = nullptr; + lstr.avail_in = 0; + lstr.next_out = nullptr; + lstr.avail_out = 0; + + // options for further compression + lzma_options_lzma opt_lzma2; + if (lzma_lzma_preset(&opt_lzma2, compression_level)) { + throw Exception(std::string("lzma preset failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + } + + lzma_filter filters[] = { + { .id = LZMA_FILTER_X86, .options = NULL }, + { .id = LZMA_FILTER_LZMA2, .options = &opt_lzma2 }, + { .id = LZMA_VLI_UNKNOWN, .options = NULL }, + }; + lzma_ret ret = lzma_stream_encoder(&lstr, filters, LZMA_CHECK_CRC64); + + if (ret != LZMA_OK) + throw Exception(std::string("lzma stream encoder init failed: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + +} + +LzmaWriteBuffer::~LzmaWriteBuffer() +{ + try { + finish(); + + lzma_end(&lstr); + } catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +void LzmaWriteBuffer::nextImpl() +{ + if (!offset()) + return; + + lstr.next_in = reinterpret_cast(working_buffer.begin()); + lstr.avail_in = offset(); + std::cout << lstr.avail_in << std::endl; + + lzma_action action = LZMA_RUN; + do { + out->nextIfAtEnd(); + lstr.next_out = reinterpret_cast(out->position()); + lstr.avail_out = out->buffer().end() - out->position(); + + lzma_ret ret = lzma_code(&lstr, action); + out->position() = out->buffer().end() - lstr.avail_out; + + if (ret == LZMA_STREAM_END) + return; + + if (ret != LZMA_OK) + throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + + std::cout << lstr.avail_in << std::endl; + } while (lstr.avail_in > 0 || lstr.avail_out == 0); +} + + +void LzmaWriteBuffer::finish() +{ + if (finished) + return; + + next(); + + do { + out->nextIfAtEnd(); + lstr.next_out = reinterpret_cast(out->position()); + lstr.avail_out = out->buffer().end() - out->position(); + + lzma_ret ret = lzma_code(&lstr, LZMA_FINISH); + out->position() = out->buffer().end() - lstr.avail_out; + + if (ret == LZMA_STREAM_END) + return; + + if (ret != LZMA_OK) + throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + + std::cout << lstr.avail_in << std::endl; + } while (lstr.avail_out == 0); + + while (true) { + out->nextIfAtEnd(); + lstr.next_out = reinterpret_cast(out->position()); + lstr.avail_out = out->buffer().end() - out->position(); + + + } +} + +} \ No newline at end of file diff --git a/src/IO/LzmaWriteBuffer.h b/src/IO/LzmaWriteBuffer.h index 6824b88b21b..986dc065c8c 100644 --- a/src/IO/LzmaWriteBuffer.h +++ b/src/IO/LzmaWriteBuffer.h @@ -1,10 +1,33 @@ #pragma once +#include +#include #include namespace DB { +/// Performs compression using lzma library and writes compressed data to out_ WriteBuffer. +class LzmaWriteBuffer : public BufferWithOwnMemory +{ +public: + LzmaWriteBuffer( + std::unique_ptr out_, + int compression_level, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + + void finish(); + + ~LzmaWriteBuffer() override; + +private: +void nextImpl() override; + +std::unique_ptr out; +lzma_stream lstr; +bool finished = false; +}; - } \ No newline at end of file diff --git a/src/IO/tests/CMakeLists.txt b/src/IO/tests/CMakeLists.txt index dfbbfa77853..b54785bf370 100644 --- a/src/IO/tests/CMakeLists.txt +++ b/src/IO/tests/CMakeLists.txt @@ -65,6 +65,9 @@ endif () add_executable (zlib_buffers zlib_buffers.cpp) target_link_libraries (zlib_buffers PRIVATE clickhouse_common_io) +add_executable (lzma_buffers lzma_buffers.cpp) +target_link_libraries (lzma_buffers PRIVATE clickhouse_common_io) + add_executable (limit_read_buffer limit_read_buffer.cpp) target_link_libraries (limit_read_buffer PRIVATE clickhouse_common_io) diff --git a/src/IO/tests/lzma_buffers.cpp b/src/IO/tests/lzma_buffers.cpp new file mode 100644 index 00000000000..6d6b1a7f2df --- /dev/null +++ b/src/IO/tests/lzma_buffers.cpp @@ -0,0 +1,66 @@ +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +int main(int, char **) +try +{ + std::cout << std::fixed << std::setprecision(2); + + size_t n = 100000; + Stopwatch stopwatch; + + { + auto buf = std::make_unique("test_lzma_buffers.xz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); + DB::LzmaWriteBuffer lzma_buf(std::move(buf), /*compression level*/ 3); + + stopwatch.restart(); + for (size_t i = 0; i < n; ++i) + { + DB::writeIntText(i, lzma_buf); + DB::writeChar('\t', lzma_buf); + } + lzma_buf.finish(); + + stopwatch.stop(); + + std::cout << "Writing done. Elapsed: " << stopwatch.elapsedSeconds() << " s." + << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" + << std::endl; + } +/* + { + auto buf = std::make_unique("test_lzma_buffers.xz"); + DB::LzmaReadBuffer lzma_buf(std::move(buf)); + + stopwatch.restart(); + for (size_t i = 0; i < n; ++i) + { + size_t x; + DB::readIntText(x, lzma_buf); + lzma_buf.ignore(); + + if (x != i) + throw DB::Exception("Failed!, read: " + std::to_string(x) + ", expected: " + std::to_string(i), 0); + } + stopwatch.stop(); + std::cout << "Reading done. Elapsed: " << stopwatch.elapsedSeconds() << " s." + << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" + << std::endl; + + } +*/ + return 0; +} +catch (const DB::Exception & e) +{ + std::cerr << e.what() << ", " << e.displayText() << std::endl; + return 1; +} From 495cd47fd8f81878a148537d86353bde22978075 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Mon, 2 Nov 2020 02:52:34 +0300 Subject: [PATCH 33/94] fixed compressor testing, added base logic for compressor and decompressor, tests passing --- src/IO/LzmaReadBuffer.cpp | 13 +++++++++---- src/IO/LzmaWriteBuffer.cpp | 33 ++++++++++++++++++--------------- src/IO/tests/lzma_buffers.cpp | 6 +++--- 3 files changed, 30 insertions(+), 22 deletions(-) diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 18cca25b4d5..3f458de3880 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -21,13 +21,13 @@ LzmaReadBuffer::LzmaReadBuffer( lstr.avail_out = 0; // 500 mb - uint64_t memlimit = 500 << 30; + uint64_t memlimit = 500 << 20; - lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED | LZMA_IGNORE_CHECK); + lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED); // lzma does not provide api for converting error code to string unlike zlib if (ret != LZMA_OK) throw Exception( - std::string("lzma_stream_decoder failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_DECODER_FAILED); } @@ -52,9 +52,10 @@ bool LzmaReadBuffer::nextImpl() lstr.next_out = reinterpret_cast(internal_buffer.begin()); lstr.avail_out = internal_buffer.size(); - lzma_ret ret = lzma_code(&lstr, LZMA_FINISH); + lzma_ret ret = lzma_code(&lstr, LZMA_RUN); in->position() = in->buffer().end() - lstr.avail_in; + working_buffer.resize(internal_buffer.size() - lstr.avail_out); if (ret == LZMA_STREAM_END) { @@ -62,6 +63,10 @@ bool LzmaReadBuffer::nextImpl() { eof = true; return working_buffer.size() != 0; + } else { + throw Exception( + std::string("lzma decoder finished, but stream is still alive: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_DECODER_FAILED); } } diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp index 6ca867d2af7..332961a292f 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LzmaWriteBuffer.cpp @@ -60,24 +60,31 @@ void LzmaWriteBuffer::nextImpl() lstr.next_in = reinterpret_cast(working_buffer.begin()); lstr.avail_in = offset(); - std::cout << lstr.avail_in << std::endl; + //std::cout << lstr.avail_in << std::endl; lzma_action action = LZMA_RUN; do { out->nextIfAtEnd(); lstr.next_out = reinterpret_cast(out->position()); lstr.avail_out = out->buffer().end() - out->position(); + //std::cout << lstr.avail_out << " BEFOR" << std::endl; + lzma_ret ret = lzma_code(&lstr, action); out->position() = out->buffer().end() - lstr.avail_out; - if (ret == LZMA_STREAM_END) - return; + //std::cout << lstr.avail_out << " AFTER" << std::endl; + + //std::cout << ret << " RET IMPL" << std::endl; + + if (ret == LZMA_STREAM_END) { + return; + } if (ret != LZMA_OK) throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - std::cout << lstr.avail_in << std::endl; + //std::cout << lstr.avail_in << " " << lstr.avail_out << std::endl; } while (lstr.avail_in > 0 || lstr.avail_out == 0); } @@ -97,22 +104,18 @@ void LzmaWriteBuffer::finish() lzma_ret ret = lzma_code(&lstr, LZMA_FINISH); out->position() = out->buffer().end() - lstr.avail_out; - if (ret == LZMA_STREAM_END) - return; + //std::cout << ret << " RET FIN" << std::endl; + + if (ret == LZMA_STREAM_END) { + finished = true; + return; + } if (ret != LZMA_OK) throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - std::cout << lstr.avail_in << std::endl; + //std::cout << lstr.avail_in << std::endl; } while (lstr.avail_out == 0); - - while (true) { - out->nextIfAtEnd(); - lstr.next_out = reinterpret_cast(out->position()); - lstr.avail_out = out->buffer().end() - out->position(); - - - } } } \ No newline at end of file diff --git a/src/IO/tests/lzma_buffers.cpp b/src/IO/tests/lzma_buffers.cpp index 6d6b1a7f2df..1f691fa09f7 100644 --- a/src/IO/tests/lzma_buffers.cpp +++ b/src/IO/tests/lzma_buffers.cpp @@ -14,7 +14,7 @@ try { std::cout << std::fixed << std::setprecision(2); - size_t n = 100000; + size_t n = 10000000; Stopwatch stopwatch; { @@ -35,7 +35,7 @@ try << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" << std::endl; } -/* + { auto buf = std::make_unique("test_lzma_buffers.xz"); DB::LzmaReadBuffer lzma_buf(std::move(buf)); @@ -56,7 +56,7 @@ try << std::endl; } -*/ + return 0; } catch (const DB::Exception & e) From 2ad01c59da53982c0b43299e272299d40928d87a Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Mon, 2 Nov 2020 16:17:25 +0300 Subject: [PATCH 34/94] fixed codestyle, added record to changelog --- CHANGELOG.md | 1 + src/IO/LzmaReadBuffer.cpp | 10 ++++------ src/IO/LzmaWriteBuffer.cpp | 30 +++++++++++++----------------- 3 files changed, 18 insertions(+), 23 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 09ce72d20ed..cc4f07ffcaa 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,7 @@ * Added column transformers `EXCEPT`, `REPLACE`, `APPLY`, which can be applied to the list of selected columns (after `*` or `COLUMNS(...)`). For example, you can write `SELECT * EXCEPT(URL) REPLACE(number + 1 AS number)`. Another example: `select * apply(length) apply(max) from wide_string_table` to find out the maxium length of all string columns. [#14233](https://github.com/ClickHouse/ClickHouse/pull/14233) ([Amos Bird](https://github.com/amosbird)). * Added an aggregate function `rankCorr` which computes a rank correlation coefficient. [#11769](https://github.com/ClickHouse/ClickHouse/pull/11769) ([antikvist](https://github.com/antikvist)) [#14411](https://github.com/ClickHouse/ClickHouse/pull/14411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Added table function `view` which turns a subquery into a table object. This helps passing queries around. For instance, it can be used in remote/cluster table functions. [#12567](https://github.com/ClickHouse/ClickHouse/pull/12567) ([Amos Bird](https://github.com/amosbird)). +* Added support for `xz` compression format. This enables using `*.xz` files in `table()` function. [#16578](https://github.com/ClickHouse/ClickHouse/pull/16578) ([Abi Palagashvili](https://github.com/fibersel)) #### Bug Fix diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 3f458de3880..7454df35a11 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -64,16 +64,14 @@ bool LzmaReadBuffer::nextImpl() eof = true; return working_buffer.size() != 0; } else { - throw Exception( - std::string("lzma decoder finished, but stream is still alive: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, - ErrorCodes::LZMA_STREAM_DECODER_FAILED); + throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", ret, LZMA_VERSION_STRING); } } if (ret != LZMA_OK) - throw Exception( - std::string("lzma_stream_decoder failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, - ErrorCodes::LZMA_STREAM_DECODER_FAILED); + throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", ret, LZMA_VERSION_STRING); return true; } diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp index 332961a292f..59b50094334 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LzmaWriteBuffer.cpp @@ -25,9 +25,9 @@ LzmaWriteBuffer::LzmaWriteBuffer( // options for further compression lzma_options_lzma opt_lzma2; - if (lzma_lzma_preset(&opt_lzma2, compression_level)) { + if (lzma_lzma_preset(&opt_lzma2, compression_level)) throw Exception(std::string("lzma preset failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - } + lzma_filter filters[] = { { .id = LZMA_FILTER_X86, .options = NULL }, @@ -43,7 +43,8 @@ LzmaWriteBuffer::LzmaWriteBuffer( LzmaWriteBuffer::~LzmaWriteBuffer() { - try { + try + { finish(); lzma_end(&lstr); @@ -60,31 +61,26 @@ void LzmaWriteBuffer::nextImpl() lstr.next_in = reinterpret_cast(working_buffer.begin()); lstr.avail_in = offset(); - //std::cout << lstr.avail_in << std::endl; lzma_action action = LZMA_RUN; - do { + do + { out->nextIfAtEnd(); lstr.next_out = reinterpret_cast(out->position()); lstr.avail_out = out->buffer().end() - out->position(); - //std::cout << lstr.avail_out << " BEFOR" << std::endl; lzma_ret ret = lzma_code(&lstr, action); out->position() = out->buffer().end() - lstr.avail_out; - //std::cout << lstr.avail_out << " AFTER" << std::endl; - //std::cout << ret << " RET IMPL" << std::endl; - if (ret == LZMA_STREAM_END) { - return; - } + if (ret == LZMA_STREAM_END) + return; if (ret != LZMA_OK) throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - //std::cout << lstr.avail_in << " " << lstr.avail_out << std::endl; } while (lstr.avail_in > 0 || lstr.avail_out == 0); } @@ -96,7 +92,8 @@ void LzmaWriteBuffer::finish() next(); - do { + do + { out->nextIfAtEnd(); lstr.next_out = reinterpret_cast(out->position()); lstr.avail_out = out->buffer().end() - out->position(); @@ -104,9 +101,9 @@ void LzmaWriteBuffer::finish() lzma_ret ret = lzma_code(&lstr, LZMA_FINISH); out->position() = out->buffer().end() - lstr.avail_out; - //std::cout << ret << " RET FIN" << std::endl; - if (ret == LZMA_STREAM_END) { + if (ret == LZMA_STREAM_END) + { finished = true; return; } @@ -114,8 +111,7 @@ void LzmaWriteBuffer::finish() if (ret != LZMA_OK) throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - //std::cout << lstr.avail_in << std::endl; } while (lstr.avail_out == 0); } -} \ No newline at end of file +} From ba6fa5d82895e20a76c12717a8a8f9ea622bc58d Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Mon, 2 Nov 2020 23:04:49 +0300 Subject: [PATCH 35/94] fixed whitespaces, added hidden submodule file --- contrib/xz | 1 + src/IO/LzmaReadBuffer.cpp | 32 ++++++++++++---------- src/IO/LzmaReadBuffer.h | 1 - src/IO/LzmaWriteBuffer.cpp | 55 +++++++++++++++++++------------------- src/IO/LzmaWriteBuffer.h | 27 +++++++++---------- 5 files changed, 60 insertions(+), 56 deletions(-) create mode 160000 contrib/xz diff --git a/contrib/xz b/contrib/xz new file mode 160000 index 00000000000..869b9d1b4ed --- /dev/null +++ b/contrib/xz @@ -0,0 +1 @@ +Subproject commit 869b9d1b4edd6df07f819d360d306251f8147353 diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 7454df35a11..377b68a4ae3 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -6,11 +6,7 @@ namespace ErrorCodes { extern const int LZMA_STREAM_DECODER_FAILED; } -LzmaReadBuffer::LzmaReadBuffer( - std::unique_ptr in_, - size_t buf_size, - char * existing_memory, - size_t alignment) +LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)) { lstr = LZMA_STREAM_INIT; @@ -27,7 +23,8 @@ LzmaReadBuffer::LzmaReadBuffer( // lzma does not provide api for converting error code to string unlike zlib if (ret != LZMA_OK) throw Exception( - std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_DECODER_FAILED); } @@ -39,9 +36,8 @@ LzmaReadBuffer::~LzmaReadBuffer() bool LzmaReadBuffer::nextImpl() { if (eof) - { return false; - } + if (!lstr.avail_in) { @@ -63,16 +59,24 @@ bool LzmaReadBuffer::nextImpl() { eof = true; return working_buffer.size() != 0; - } else { - throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", ret, LZMA_VERSION_STRING); + } + else + { + throw Exception( + ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); } } if (ret != LZMA_OK) - throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", ret, LZMA_VERSION_STRING); + throw Exception( + ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); return true; } -} \ No newline at end of file +} diff --git a/src/IO/LzmaReadBuffer.h b/src/IO/LzmaReadBuffer.h index 3ece8a46b7d..ae4d4ccf749 100644 --- a/src/IO/LzmaReadBuffer.h +++ b/src/IO/LzmaReadBuffer.h @@ -30,5 +30,4 @@ private: lzma_stream lstr; bool eof; }; - } \ No newline at end of file diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp index 59b50094334..395de00be1d 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LzmaWriteBuffer.cpp @@ -1,20 +1,16 @@ #include -namespace DB { +namespace DB +{ namespace ErrorCodes { extern const int LZMA_STREAM_ENCODER_FAILED; } LzmaWriteBuffer::LzmaWriteBuffer( - std::unique_ptr out_, - int compression_level, - size_t buf_size, - char * existing_memory, - size_t alignment) - : BufferWithOwnMemory(buf_size, existing_memory, alignment) - , out(std::move(out_)) + std::unique_ptr out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment), out(std::move(out_)) { lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; @@ -22,23 +18,25 @@ LzmaWriteBuffer::LzmaWriteBuffer( lstr.avail_in = 0; lstr.next_out = nullptr; lstr.avail_out = 0; - + // options for further compression lzma_options_lzma opt_lzma2; if (lzma_lzma_preset(&opt_lzma2, compression_level)) - throw Exception(std::string("lzma preset failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + throw Exception( + std::string("lzma preset failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); lzma_filter filters[] = { - { .id = LZMA_FILTER_X86, .options = NULL }, - { .id = LZMA_FILTER_LZMA2, .options = &opt_lzma2 }, - { .id = LZMA_VLI_UNKNOWN, .options = NULL }, + {.id = LZMA_FILTER_X86, .options = NULL}, + {.id = LZMA_FILTER_LZMA2, .options = &opt_lzma2}, + {.id = LZMA_VLI_UNKNOWN, .options = NULL}, }; lzma_ret ret = lzma_stream_encoder(&lstr, filters, LZMA_CHECK_CRC64); - if (ret != LZMA_OK) - throw Exception(std::string("lzma stream encoder init failed: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - + if (ret != LZMA_OK) + throw Exception( + std::string("lzma stream encoder init failed: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_ENCODER_FAILED); } LzmaWriteBuffer::~LzmaWriteBuffer() @@ -48,7 +46,8 @@ LzmaWriteBuffer::~LzmaWriteBuffer() finish(); lzma_end(&lstr); - } catch (...) + } + catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -74,13 +73,14 @@ void LzmaWriteBuffer::nextImpl() out->position() = out->buffer().end() - lstr.avail_out; - if (ret == LZMA_STREAM_END) return; - + if (ret != LZMA_OK) - throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - + throw Exception( + std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + } while (lstr.avail_in > 0 || lstr.avail_out == 0); } @@ -102,16 +102,17 @@ void LzmaWriteBuffer::finish() out->position() = out->buffer().end() - lstr.avail_out; - if (ret == LZMA_STREAM_END) + if (ret == LZMA_STREAM_END) { finished = true; - return; + return; } - + if (ret != LZMA_OK) - throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - + throw Exception( + std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + } while (lstr.avail_out == 0); } - } diff --git a/src/IO/LzmaWriteBuffer.h b/src/IO/LzmaWriteBuffer.h index 986dc065c8c..fa5e3456f79 100644 --- a/src/IO/LzmaWriteBuffer.h +++ b/src/IO/LzmaWriteBuffer.h @@ -1,33 +1,32 @@ #pragma once -#include #include +#include #include -namespace DB { - +namespace DB +{ /// Performs compression using lzma library and writes compressed data to out_ WriteBuffer. class LzmaWriteBuffer : public BufferWithOwnMemory -{ +{ public: LzmaWriteBuffer( - std::unique_ptr out_, - int compression_level, - size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, - char * existing_memory = nullptr, - size_t alignment = 0); + std::unique_ptr out_, + int compression_level, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); void finish(); ~LzmaWriteBuffer() override; private: -void nextImpl() override; + void nextImpl() override; -std::unique_ptr out; -lzma_stream lstr; -bool finished = false; + std::unique_ptr out; + lzma_stream lstr; + bool finished = false; }; - } \ No newline at end of file From 8098f86e3ea5bcefb84601ed0ecd3de5096e022a Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Tue, 3 Nov 2020 01:05:50 +0300 Subject: [PATCH 36/94] added record for fasttest --- docker/test/fasttest/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 8300c31681e..add8d2254f9 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/xz) git submodule sync git submodule update --init --recursive "${SUBMODULES_TO_UPDATE[@]}" From 8ecf1d07c1a1791485ca9a49d01f7327db4f0b9a Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 4 Nov 2020 01:37:05 +0300 Subject: [PATCH 37/94] attempt to update fasttest version --- docker/test/fasttest/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index f6c665ff3fd..ac22a9dfaf0 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -1,5 +1,5 @@ # docker build -t yandex/clickhouse-fasttest . -FROM ubuntu:19.10 +FROM ubuntu:20.04 ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=10 From f9cebbfa81b6bb50a9f67d35574b08680f76a7e7 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 4 Nov 2020 02:47:39 +0300 Subject: [PATCH 38/94] added newline for files --- src/IO/LzmaReadBuffer.h | 2 +- src/IO/LzmaWriteBuffer.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/LzmaReadBuffer.h b/src/IO/LzmaReadBuffer.h index ae4d4ccf749..5f936475ee1 100644 --- a/src/IO/LzmaReadBuffer.h +++ b/src/IO/LzmaReadBuffer.h @@ -30,4 +30,4 @@ private: lzma_stream lstr; bool eof; }; -} \ No newline at end of file +} diff --git a/src/IO/LzmaWriteBuffer.h b/src/IO/LzmaWriteBuffer.h index fa5e3456f79..d59595dab23 100644 --- a/src/IO/LzmaWriteBuffer.h +++ b/src/IO/LzmaWriteBuffer.h @@ -29,4 +29,4 @@ private: lzma_stream lstr; bool finished = false; }; -} \ No newline at end of file +} From 986d13df3a7df63065d8eff084b174072673ac22 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 4 Nov 2020 03:35:16 +0300 Subject: [PATCH 39/94] replaced null with nullptr --- src/IO/LzmaWriteBuffer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp index 395de00be1d..8439fc624d4 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LzmaWriteBuffer.cpp @@ -27,9 +27,9 @@ LzmaWriteBuffer::LzmaWriteBuffer( lzma_filter filters[] = { - {.id = LZMA_FILTER_X86, .options = NULL}, + {.id = LZMA_FILTER_X86, .options = nullptr}, {.id = LZMA_FILTER_LZMA2, .options = &opt_lzma2}, - {.id = LZMA_VLI_UNKNOWN, .options = NULL}, + {.id = LZMA_VLI_UNKNOWN, .options = nullptr}, }; lzma_ret ret = lzma_stream_encoder(&lstr, filters, LZMA_CHECK_CRC64); From 53a064b6e38737154f2eccab3336e1db0b7e36ad Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 4 Nov 2020 15:45:37 +0300 Subject: [PATCH 40/94] added eof initializing in constuctor --- src/IO/LzmaReadBuffer.cpp | 53 ++++++++++++++++----------------------- 1 file changed, 21 insertions(+), 32 deletions(-) diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 377b68a4ae3..1763a101694 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -1,13 +1,11 @@ #include -namespace DB -{ -namespace ErrorCodes -{ +namespace DB { +namespace ErrorCodes { extern const int LZMA_STREAM_DECODER_FAILED; } -LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) - : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)) +LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char* existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)), eof(false) { lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; @@ -22,10 +20,9 @@ LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED); // lzma does not provide api for converting error code to string unlike zlib if (ret != LZMA_OK) - throw Exception( - std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) - + "; lzma version: " + LZMA_VERSION_STRING, - ErrorCodes::LZMA_STREAM_DECODER_FAILED); + throw Exception(std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + "; lzma version: " + + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_DECODER_FAILED); } LzmaReadBuffer::~LzmaReadBuffer() @@ -38,14 +35,12 @@ bool LzmaReadBuffer::nextImpl() if (eof) return false; - - if (!lstr.avail_in) - { + if (!lstr.avail_in) { in->nextIfAtEnd(); - lstr.next_in = reinterpret_cast(in->position()); + lstr.next_in = reinterpret_cast(in->position()); lstr.avail_in = in->buffer().end() - in->position(); } - lstr.next_out = reinterpret_cast(internal_buffer.begin()); + lstr.next_out = reinterpret_cast(internal_buffer.begin()); lstr.avail_out = internal_buffer.size(); lzma_ret ret = lzma_code(&lstr, LZMA_RUN); @@ -53,29 +48,23 @@ bool LzmaReadBuffer::nextImpl() in->position() = in->buffer().end() - lstr.avail_in; working_buffer.resize(internal_buffer.size() - lstr.avail_out); - if (ret == LZMA_STREAM_END) - { - if (in->eof()) - { + if (ret == LZMA_STREAM_END) { + if (in->eof()) { eof = true; return working_buffer.size() != 0; - } - else - { - throw Exception( - ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", - ret, - LZMA_VERSION_STRING); + } else { + throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); } } if (ret != LZMA_OK) - throw Exception( - ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", - ret, - LZMA_VERSION_STRING); + throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); return true; } From 87cc3546caa601930f67a463c5abccf53551a021 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 4 Nov 2020 19:39:26 +0300 Subject: [PATCH 41/94] fix codestyle --- src/IO/LzmaReadBuffer.cpp | 50 +++++++++++++++++++++++---------------- 1 file changed, 30 insertions(+), 20 deletions(-) diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 1763a101694..22fda48b3c6 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -1,10 +1,12 @@ #include -namespace DB { -namespace ErrorCodes { +namespace DB +{ +namespace ErrorCodes +{ extern const int LZMA_STREAM_DECODER_FAILED; } -LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char* existing_memory, size_t alignment) +LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)), eof(false) { lstr = LZMA_STREAM_INIT; @@ -20,9 +22,10 @@ LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED); // lzma does not provide api for converting error code to string unlike zlib if (ret != LZMA_OK) - throw Exception(std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + "; lzma version: " - + LZMA_VERSION_STRING, - ErrorCodes::LZMA_STREAM_DECODER_FAILED); + throw Exception( + std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_DECODER_FAILED); } LzmaReadBuffer::~LzmaReadBuffer() @@ -35,12 +38,13 @@ bool LzmaReadBuffer::nextImpl() if (eof) return false; - if (!lstr.avail_in) { + if (!lstr.avail_in) + { in->nextIfAtEnd(); - lstr.next_in = reinterpret_cast(in->position()); + lstr.next_in = reinterpret_cast(in->position()); lstr.avail_in = in->buffer().end() - in->position(); } - lstr.next_out = reinterpret_cast(internal_buffer.begin()); + lstr.next_out = reinterpret_cast(internal_buffer.begin()); lstr.avail_out = internal_buffer.size(); lzma_ret ret = lzma_code(&lstr, LZMA_RUN); @@ -48,23 +52,29 @@ bool LzmaReadBuffer::nextImpl() in->position() = in->buffer().end() - lstr.avail_in; working_buffer.resize(internal_buffer.size() - lstr.avail_out); - if (ret == LZMA_STREAM_END) { - if (in->eof()) { + if (ret == LZMA_STREAM_END) + { + if (in->eof()) + { eof = true; return working_buffer.size() != 0; - } else { - throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", - ret, - LZMA_VERSION_STRING); + } + else + { + throw Exception( + ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); } } if (ret != LZMA_OK) - throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", - ret, - LZMA_VERSION_STRING); + throw Exception( + ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); return true; } From 73e5d283aa472ce897e9df57be32265ff686cc9d Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Sat, 7 Nov 2020 16:24:21 +0300 Subject: [PATCH 42/94] regenerated ya.make --- src/IO/ya.make | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ya.make b/src/IO/ya.make index 3796494ff33..f3b14a94a94 100644 --- a/src/IO/ya.make +++ b/src/IO/ya.make @@ -24,6 +24,8 @@ SRCS( HexWriteBuffer.cpp HTTPCommon.cpp LimitReadBuffer.cpp + LzmaReadBuffer.cpp + LzmaWriteBuffer.cpp MemoryReadWriteBuffer.cpp MMapReadBufferFromFile.cpp MMapReadBufferFromFileDescriptor.cpp From a06be511df8ebd0a963470026d8460257b891dc7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 9 Nov 2020 16:07:38 +0300 Subject: [PATCH 43/94] pcg serialization --- base/pcg-random/pcg_random.hpp | 9 +++++++ .../AggregateFunctionGroupArray.h | 27 +++++++++---------- src/AggregateFunctions/ReservoirSampler.h | 15 ++++++----- src/Client/MultiplexedConnections.cpp | 8 +++--- src/IO/Operators.h | 2 ++ src/IO/ReadHelpers.h | 20 ++++++++++++++ src/IO/WriteHelpers.h | 12 +++++++++ utils/check-style/check-style | 3 +++ 8 files changed, 71 insertions(+), 25 deletions(-) diff --git a/base/pcg-random/pcg_random.hpp b/base/pcg-random/pcg_random.hpp index d96d5895b31..f7778480c4c 100644 --- a/base/pcg-random/pcg_random.hpp +++ b/base/pcg-random/pcg_random.hpp @@ -113,6 +113,12 @@ #include "pcg_extras.hpp" +namespace DB +{ + struct PcgSerializer; + struct PcgDeserializer; +} + namespace pcg_detail { using namespace pcg_extras; @@ -557,6 +563,9 @@ public: engine& rng); + + friend ::DB::PcgSerializer; + friend ::DB::PcgDeserializer; }; template #include +#include +#include +#include #include #include @@ -244,10 +247,9 @@ public: if constexpr (Trait::sampler == Sampler::RNG) { DB::writeIntBinary(this->data(place).total_values, buf); - std::ostringstream rng_stream; - rng_stream.exceptions(std::ios::failbit); - rng_stream << this->data(place).rng; - DB::writeStringBinary(rng_stream.str(), buf); + WriteBufferFromOwnString rng_buf; + rng_buf << this->data(place).rng; + DB::writeStringBinary(rng_buf.str(), buf); } // TODO @@ -275,9 +277,8 @@ public: DB::readIntBinary(this->data(place).total_values, buf); std::string rng_string; DB::readStringBinary(rng_string, buf); - std::istringstream rng_stream(rng_string); - rng_stream.exceptions(std::ios::failbit); - rng_stream >> this->data(place).rng; + ReadBufferFromString rng_buf(rng_string); + rng_buf >> this->data(place).rng; } // TODO @@ -565,10 +566,9 @@ public: if constexpr (Trait::sampler == Sampler::RNG) { DB::writeIntBinary(data(place).total_values, buf); - std::ostringstream rng_stream; - rng_stream.exceptions(std::ios::failbit); - rng_stream << data(place).rng; - DB::writeStringBinary(rng_stream.str(), buf); + WriteBufferFromOwnString rng_buf; + rng_buf << data(place).rng; + DB::writeStringBinary(rng_buf.str(), buf); } // TODO @@ -600,9 +600,8 @@ public: DB::readIntBinary(data(place).total_values, buf); std::string rng_string; DB::readStringBinary(rng_string, buf); - std::istringstream rng_stream(rng_string); - rng_stream.exceptions(std::ios::failbit); - rng_stream >> data(place).rng; + ReadBufferFromString rng_buf(rng_string); + rng_buf >> data(place).rng; } // TODO diff --git a/src/AggregateFunctions/ReservoirSampler.h b/src/AggregateFunctions/ReservoirSampler.h index f82b0b856a0..c5d2158d59c 100644 --- a/src/AggregateFunctions/ReservoirSampler.h +++ b/src/AggregateFunctions/ReservoirSampler.h @@ -8,6 +8,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -190,9 +193,8 @@ public: std::string rng_string; DB::readStringBinary(rng_string, buf); - std::istringstream rng_stream(rng_string); - rng_stream.exceptions(std::ios::failbit); - rng_stream >> rng; + DB::ReadBufferFromString rng_buf(rng_string); + rng_buf >> rng; for (size_t i = 0; i < samples.size(); ++i) DB::readBinary(samples[i], buf); @@ -205,10 +207,9 @@ public: DB::writeIntBinary(sample_count, buf); DB::writeIntBinary(total_values, buf); - std::ostringstream rng_stream; - rng_stream.exceptions(std::ios::failbit); - rng_stream << rng; - DB::writeStringBinary(rng_stream.str(), buf); + DB::WriteBufferFromOwnString rng_buf; + rng_buf << rng; + DB::writeStringBinary(rng_buf.str(), buf); for (size_t i = 0; i < std::min(sample_count, total_values); ++i) DB::writeBinary(samples[i], buf); diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index b6cb55fa979..a99b0f9d7cc 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -222,19 +223,18 @@ std::string MultiplexedConnections::dumpAddresses() const std::string MultiplexedConnections::dumpAddressesUnlocked() const { bool is_first = true; - std::ostringstream os; - os.exceptions(std::ios::failbit); + WriteBufferFromOwnString buf; for (const ReplicaState & state : replica_states) { const Connection * connection = state.connection; if (connection) { - os << (is_first ? "" : "; ") << connection->getDescription(); + buf << (is_first ? "" : "; ") << connection->getDescription(); is_first = false; } } - return os.str(); + return buf.str(); } Packet MultiplexedConnections::receivePacketUnlocked() diff --git a/src/IO/Operators.h b/src/IO/Operators.h index 7e276ba9248..bd0b1de8fad 100644 --- a/src/IO/Operators.h +++ b/src/IO/Operators.h @@ -46,6 +46,7 @@ template WriteBuffer & operator<< (WriteBuffer & buf, const T & /// If you do not use the manipulators, the string is displayed without an escape, as is. template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const String & x) { writeString(x, buf); return buf; } template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const char & x) { writeChar(x, buf); return buf; } +template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const pcg32_fast & x) { PcgSerializer::serializePcg32(x, buf); return buf; } inline WriteBuffer & operator<< (WriteBuffer & buf, const char * x) { writeCString(x, buf); return buf; } @@ -73,6 +74,7 @@ inline WriteBuffer & operator<< (WriteBuffer & buf, FlushManip) { buf.next(); re template ReadBuffer & operator>> (ReadBuffer & buf, T & x) { readText(x, buf); return buf; } template <> inline ReadBuffer & operator>> (ReadBuffer & buf, String & x) { readString(x, buf); return buf; } template <> inline ReadBuffer & operator>> (ReadBuffer & buf, char & x) { readChar(x, buf); return buf; } +template <> inline ReadBuffer & operator>> (ReadBuffer & buf, pcg32_fast & x) { PcgDeserializer::deserializePcg32(x, buf); return buf; } /// If you specify a string literal for reading, this will mean - make sure there is a sequence of bytes and skip it. inline ReadBuffer & operator>> (ReadBuffer & buf, const char * x) { assertString(x, buf); return buf; } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 9ff1858c723..56ed4d20cef 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -53,6 +53,7 @@ namespace ErrorCodes extern const int CANNOT_READ_ARRAY_FROM_TEXT; extern const int CANNOT_PARSE_NUMBER; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int INCORRECT_DATA; } /// Helper functions for formatted input. @@ -1228,4 +1229,23 @@ void saveUpToPosition(ReadBuffer & in, Memory<> & memory, char * current); */ bool loadAtPosition(ReadBuffer & in, Memory<> & memory, char * & current); + +struct PcgDeserializer +{ + static void deserializePcg32(const pcg32_fast & rng, ReadBuffer & buf) + { + decltype(rng.state_) multiplier, increment, state; + readText(multiplier, buf); + assertChar(' ', buf); + readText(increment, buf); + assertChar(' ', buf); + readText(state, buf); + + if (multiplier != rng.multiplier()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect multiplier in pcg32: expected {}, got {}", rng.multiplier(), multiplier); + if (increment != rng.increment()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect increment in pcg32: expected {}, got {}", rng.increment(), increment); + } +}; + } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 9d517913582..a1d74e2fa6a 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1093,4 +1093,16 @@ writeBinaryBigEndian(T x, WriteBuffer & buf) /// Assuming little endian archi writePODBinary(x, buf); } +struct PcgSerializer +{ + static void serializePcg32(const pcg32_fast & rng, WriteBuffer & buf) + { + writeText(rng.multiplier(), buf); + writeChar(' ', buf); + writeText(rng.increment(), buf); + writeChar(' ', buf); + writeText(rng.state_, buf); + } +}; + } diff --git a/utils/check-style/check-style b/utils/check-style/check-style index f9818a1e2bb..69d948c87fe 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -105,3 +105,6 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs # Trailing whitespaces find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P ' $' | grep -P '.' && echo "^ Trailing whitespaces." + +# Forbid stringstream because it's easy to use them incorrectly and hard to debug possible issues +find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep 'std::ostringstream\|std::istringstream' && echo "Use WriteBufferFromString or ReadBufferFromString instead of std::ostringstream or std::istringstream" 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 44/94] 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 45/94] 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 62ff00ee8b2df2ce331e8cf4771d1bac9493c276 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 9 Nov 2020 19:05:40 +0300 Subject: [PATCH 46/94] use WriteBuffer in formatAST(...) --- programs/client/Client.cpp | 4 ++- programs/client/QueryFuzzer.cpp | 4 ++- programs/format/Format.cpp | 4 ++- programs/odbc-bridge/ColumnInfoHandler.cpp | 6 ++--- .../odbc-bridge/ODBCBlockOutputStream.cpp | 6 ++--- src/Access/DiskAccessStorage.cpp | 10 ++++--- src/Databases/DatabaseOnDisk.cpp | 16 +++++------- src/IO/Operators.h | 1 + src/IO/WriteHelpers.h | 5 ++++ src/Interpreters/AddDefaultDatabaseVisitor.h | 4 +-- .../ClusterProxy/SelectStreamFactory.cpp | 7 +++-- src/Interpreters/InDepthNodeVisitor.h | 4 +-- src/Interpreters/InterpreterCreateQuery.cpp | 14 ++++------ src/Interpreters/InterpreterExplainQuery.cpp | 20 ++++++-------- .../InterpreterShowAccessQuery.cpp | 9 +++---- ...InterpreterShowCreateAccessEntityQuery.cpp | 16 +++++------- .../InterpreterShowCreateQuery.cpp | 7 +++-- .../InterpreterShowGrantsQuery.cpp | 16 +++++------- src/Interpreters/QueryAliasesVisitor.cpp | 9 +++---- src/Parsers/ASTAlterQuery.cpp | 7 ++--- src/Parsers/ASTAsterisk.cpp | 1 + src/Parsers/ASTColumnDeclaration.cpp | 1 + src/Parsers/ASTColumnsMatcher.cpp | 1 + src/Parsers/ASTColumnsTransformers.cpp | 1 + src/Parsers/ASTConstraintDeclaration.cpp | 1 + src/Parsers/ASTCreateQuery.cpp | 1 + src/Parsers/ASTCreateQuotaQuery.cpp | 1 + src/Parsers/ASTCreateRoleQuery.cpp | 1 + src/Parsers/ASTCreateRowPolicyQuery.cpp | 10 +++---- src/Parsers/ASTCreateSettingsProfileQuery.cpp | 1 + src/Parsers/ASTCreateUserQuery.cpp | 1 + src/Parsers/ASTDictionary.cpp | 1 + .../ASTDictionaryAttributeDeclaration.cpp | 1 + src/Parsers/ASTDropAccessEntityQuery.cpp | 1 + src/Parsers/ASTDropQuery.cpp | 1 + src/Parsers/ASTExpressionList.cpp | 1 + src/Parsers/ASTFunction.cpp | 1 + .../ASTFunctionWithKeyValueArguments.cpp | 1 + src/Parsers/ASTGrantQuery.cpp | 1 + src/Parsers/ASTIdentifier.cpp | 1 + src/Parsers/ASTIndexDeclaration.cpp | 1 + src/Parsers/ASTInsertQuery.cpp | 1 + src/Parsers/ASTKillQueryQuery.cpp | 1 + src/Parsers/ASTLiteral.cpp | 5 ++++ src/Parsers/ASTLiteral.h | 5 +--- src/Parsers/ASTNameTypePair.cpp | 1 + src/Parsers/ASTOptimizeQuery.cpp | 1 + src/Parsers/ASTOrderByElement.cpp | 1 + src/Parsers/ASTPartition.cpp | 1 + src/Parsers/ASTQualifiedAsterisk.cpp | 1 + src/Parsers/ASTQueryParameter.cpp | 1 + src/Parsers/ASTQueryWithOnCluster.cpp | 1 + src/Parsers/ASTQueryWithOutput.h | 1 + src/Parsers/ASTQueryWithTableAndOutput.cpp | 1 + src/Parsers/ASTRenameQuery.h | 1 + src/Parsers/ASTRolesOrUsersSet.cpp | 1 + src/Parsers/ASTRowPolicyName.cpp | 1 + src/Parsers/ASTSampleRatio.cpp | 5 ++++ src/Parsers/ASTSampleRatio.h | 5 +--- src/Parsers/ASTSelectQuery.cpp | 1 + src/Parsers/ASTSelectWithUnionQuery.cpp | 1 + src/Parsers/ASTSetQuery.cpp | 1 + src/Parsers/ASTSetRoleQuery.cpp | 1 + src/Parsers/ASTSettingsProfileElement.cpp | 1 + src/Parsers/ASTShowAccessEntitiesQuery.cpp | 1 + .../ASTShowCreateAccessEntityQuery.cpp | 1 + src/Parsers/ASTShowGrantsQuery.cpp | 1 + src/Parsers/ASTShowPrivilegesQuery.cpp | 0 src/Parsers/ASTShowTablesQuery.cpp | 3 ++- src/Parsers/ASTSubquery.cpp | 1 + src/Parsers/ASTSystemQuery.cpp | 1 + src/Parsers/ASTTTLElement.cpp | 1 + src/Parsers/ASTTablesInSelectQuery.cpp | 1 + src/Parsers/ASTUseQuery.h | 1 + src/Parsers/ASTUserNameWithHost.cpp | 1 + src/Parsers/ASTWithAlias.cpp | 1 + src/Parsers/ASTWithElement.cpp | 1 + src/Parsers/CommonParsers.cpp | 1 + src/Parsers/DumpASTNode.h | 22 ++++++++-------- src/Parsers/IAST.cpp | 25 +++++++----------- src/Parsers/IAST.h | 17 ++++++------ src/Parsers/formatAST.cpp | 11 ++++---- src/Parsers/formatAST.h | 26 ++++++++++--------- src/Parsers/formatSettingName.cpp | 4 +-- src/Parsers/formatSettingName.h | 4 ++- src/Parsers/queryToString.cpp | 6 +---- src/Parsers/tests/create_parser.cpp | 4 ++- src/Parsers/tests/gtest_dictionary_parser.cpp | 8 +++--- src/Parsers/tests/select_parser.cpp | 4 ++- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 7 +++-- .../ReplicatedMergeTreeTableMetadata.cpp | 7 +++-- src/Storages/MutationCommands.cpp | 7 +++-- src/Storages/StorageMergeTree.cpp | 7 +++-- .../transformQueryForExternalDatabase.cpp | 3 +-- utils/db-generator/query_db_generator.cpp | 19 +++++++------- 95 files changed, 241 insertions(+), 193 deletions(-) delete mode 100644 src/Parsers/ASTShowPrivilegesQuery.cpp diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index fc452c165e7..f9e4b357aa9 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -54,6 +54,7 @@ #include #include #include +#include #include #include #include @@ -1529,7 +1530,8 @@ private: if (is_interactive) { std::cout << std::endl; - formatAST(*res, std::cout); + WriteBufferFromOStream res_buf(std::cout, 4096); + formatAST(*res, res_buf); std::cout << std::endl << std::endl; } diff --git a/programs/client/QueryFuzzer.cpp b/programs/client/QueryFuzzer.cpp index 3fe6b8087f0..cb058f6db00 100644 --- a/programs/client/QueryFuzzer.cpp +++ b/programs/client/QueryFuzzer.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -419,7 +420,8 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast) fuzz(ast); std::cout << std::endl; - formatAST(*ast, std::cout, false /*highlight*/); + WriteBufferFromOStream ast_buf(std::cout, 4096); + formatAST(*ast, ast_buf, false /*highlight*/); std::cout << std::endl << std::endl; } diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 01f952bf95e..c9981e706b0 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -129,7 +130,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv) ASTPtr res = parseQueryAndMovePosition(parser, pos, end, "query", multiple, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); if (!quiet) { - formatAST(*res, std::cout, hilite, oneline); + WriteBufferFromOStream res_buf(std::cout, 4096); + formatAST(*res, res_buf, hilite, oneline); if (multiple) std::cout << "\n;\n"; std::cout << std::endl; diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 33723b4cc8e..9e2307715c5 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -113,16 +113,16 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques /// TODO Why not do SQLColumns instead? std::string name = schema_name.empty() ? backQuoteIfNeed(table_name) : backQuoteIfNeed(schema_name) + "." + backQuoteIfNeed(table_name); - std::stringstream ss; + WriteBufferFromOwnString buf; std::string input = "SELECT * FROM " + name + " WHERE 1 = 0"; ParserQueryWithOutput parser; ASTPtr select = parseQuery(parser, input.data(), input.data() + input.size(), "", context_settings.max_query_size, context_settings.max_parser_depth); - IAST::FormatSettings settings(ss, true); + IAST::FormatSettings settings(buf, true); settings.always_quote_identifiers = true; settings.identifier_quoting_style = getQuotingStyle(hdbc); select->format(settings); - std::string query = ss.str(); + std::string query = buf.str(); LOG_TRACE(log, "Inferring structure with query '{}'", query); diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index 82ca861ea67..4d8b9fa6bdf 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -32,12 +32,12 @@ namespace for (const auto & column : columns) query.columns->children.emplace_back(std::make_shared(column.name)); - std::stringstream ss; - IAST::FormatSettings settings(ss, true); + WriteBufferFromOwnString buf; + IAST::FormatSettings settings(buf, true); settings.always_quote_identifiers = true; settings.identifier_quoting_style = quoting; query.IAST::format(settings); - return ss.str(); + return buf.str(); } std::string getQuestionMarks(size_t n) diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index abf4ff12d5a..b03a2791560 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -197,11 +197,13 @@ namespace boost::range::push_back(queries, InterpreterShowGrantsQuery::getAttachGrantQueries(entity)); /// Serialize the list of ATTACH queries to a string. - std::stringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString buf; for (const ASTPtr & query : queries) - ss << *query << ";\n"; - String file_contents = std::move(ss).str(); + { + formatAST(*query, buf, false, true); + buf.write(";\n", 2); + } + String file_contents = buf.str(); /// First we save *.tmp file and then we rename if everything's ok. auto tmp_file_path = std::filesystem::path{file_path}.replace_extension(".tmp"); diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 83e70a25f87..1e6b4019c4b 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -94,10 +94,9 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) if (!create) { - std::ostringstream query_stream; - query_stream.exceptions(std::ios::failbit); - formatAST(*query, query_stream, true); - throw Exception("Query '" + query_stream.str() + "' is not CREATE query", ErrorCodes::LOGICAL_ERROR); + WriteBufferFromOwnString query_buf; + formatAST(*query, query_buf, true); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query '{}' is not CREATE query", query_buf.str()); } if (!create->is_dictionary) @@ -121,11 +120,10 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) if (create->uuid != UUIDHelpers::Nil) create->table = TABLE_WITH_UUID_NAME_PLACEHOLDER; - std::ostringstream statement_stream; - statement_stream.exceptions(std::ios::failbit); - formatAST(*create, statement_stream, false); - statement_stream << '\n'; - return statement_stream.str(); + WriteBufferFromOwnString statement_buf; + formatAST(*create, statement_buf, false); + writeChar('\n', statement_buf); + return statement_buf.str(); } DatabaseOnDisk::DatabaseOnDisk( diff --git a/src/IO/Operators.h b/src/IO/Operators.h index bd0b1de8fad..d1500aedd22 100644 --- a/src/IO/Operators.h +++ b/src/IO/Operators.h @@ -45,6 +45,7 @@ struct BinaryManipReadBuffer : std::reference_wrapper { usin template WriteBuffer & operator<< (WriteBuffer & buf, const T & x) { writeText(x, buf); return buf; } /// If you do not use the manipulators, the string is displayed without an escape, as is. template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const String & x) { writeString(x, buf); return buf; } +template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const std::string_view & x) { writeString(StringRef(x), buf); return buf; } template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const char & x) { writeChar(x, buf); return buf; } template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const pcg32_fast & x) { PcgSerializer::serializePcg32(x, buf); return buf; } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index a1d74e2fa6a..da68c4aded5 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -271,6 +271,11 @@ inline void writeString(const StringRef & ref, WriteBuffer & buf) writeString(ref.data, ref.size, buf); } +//inline void writeString(const std::string_view & view, WriteBuffer & buf) +//{ +// writeString(view.data(), view.size(), buf); +//} + /** Writes a C-string without creating a temporary object. If the string is a literal, then `strlen` is executed at the compilation stage. * Use when the string is a literal. diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index bb684c5547a..7b72374c9c6 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -25,7 +25,7 @@ class AddDefaultDatabaseVisitor { public: explicit AddDefaultDatabaseVisitor( - const String & database_name_, bool only_replace_current_database_function_ = false, std::ostream * ostr_ = nullptr) + const String & database_name_, bool only_replace_current_database_function_ = false, WriteBuffer * ostr_ = nullptr) : database_name(database_name_) , only_replace_current_database_function(only_replace_current_database_function_) , visit_depth(0) @@ -66,7 +66,7 @@ private: const String database_name; bool only_replace_current_database_function = false; mutable size_t visit_depth; - std::ostream * ostr; + WriteBuffer * ostr; void visit(ASTSelectWithUnionQuery & select, ASTPtr &) const { diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 9e64695d1a0..f0f68c0d93f 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -106,10 +106,9 @@ String formattedAST(const ASTPtr & ast) { if (!ast) return {}; - std::stringstream ss; - ss.exceptions(std::ios::failbit); - formatAST(*ast, ss, false, true); - return ss.str(); + WriteBufferFromOwnString buf; + formatAST(*ast, buf, false, true); + return buf.str(); } } diff --git a/src/Interpreters/InDepthNodeVisitor.h b/src/Interpreters/InDepthNodeVisitor.h index 7b537f0daa0..7a793566cdd 100644 --- a/src/Interpreters/InDepthNodeVisitor.h +++ b/src/Interpreters/InDepthNodeVisitor.h @@ -16,7 +16,7 @@ class InDepthNodeVisitor public: using Data = typename Matcher::Data; - InDepthNodeVisitor(Data & data_, std::ostream * ostr_ = nullptr) + InDepthNodeVisitor(Data & data_, WriteBuffer * ostr_ = nullptr) : data(data_), visit_depth(0), ostr(ostr_) @@ -46,7 +46,7 @@ public: private: Data & data; size_t visit_depth; - std::ostream * ostr; + WriteBuffer * ostr; void visitChildren(T & ast) { diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 286d5269a64..d6e4dc666e7 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -135,10 +135,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty())) { /// Currently, there are no database engines, that support any arguments. - std::stringstream ostr; - ostr.exceptions(std::ios::failbit); - formatAST(*create.storage, ostr, false, false); - throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE); + throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", serializeAST(*create.storage)); } if (create.storage->engine->name == "Atomic") @@ -182,11 +179,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) create.attach = true; create.if_not_exists = false; - std::ostringstream statement_stream; - statement_stream.exceptions(std::ios::failbit); - formatAST(create, statement_stream, false); - statement_stream << '\n'; - String statement = statement_stream.str(); + WriteBufferFromOwnString statement_buf; + formatAST(create, statement_buf, false); + writeChar('\n', statement_buf); + String statement = statement_buf.str(); /// Exclusive flag guarantees, that database is not created right now in another thread. WriteBufferFromFile out(metadata_file_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL); diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index ed791f0d592..ae8e2637cbb 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -222,15 +222,14 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() Block sample_block = getSampleBlock(); MutableColumns res_columns = sample_block.cloneEmptyColumns(); - std::stringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString buf; if (ast.getKind() == ASTExplainQuery::ParsedAST) { if (ast.getSettings()) throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING); - dumpAST(*ast.getExplainedQuery(), ss); + dumpAST(*ast.getExplainedQuery(), buf); } else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax) { @@ -240,7 +239,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() ExplainAnalyzedSyntaxVisitor::Data data{.context = context}; ExplainAnalyzedSyntaxVisitor(data).visit(query); - ast.getExplainedQuery()->format(IAST::FormatSettings(ss, false)); + ast.getExplainedQuery()->format(IAST::FormatSettings(buf, false)); } else if (ast.getKind() == ASTExplainQuery::QueryPlan) { @@ -256,8 +255,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() if (settings.optimize) plan.optimize(); - WriteBufferFromOStream buffer(ss); - plan.explainPlan(buffer, settings.query_plan_options); + plan.explainPlan(buf, settings.query_plan_options); } else if (ast.getKind() == ASTExplainQuery::QueryPipeline) { @@ -271,8 +269,6 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() interpreter.buildQueryPlan(plan); auto pipeline = plan.buildQueryPipeline(); - WriteBufferFromOStream buffer(ss); - if (settings.graph) { /// Pipe holds QueryPlan, should not go out-of-scope @@ -280,17 +276,17 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() const auto & processors = pipe.getProcessors(); if (settings.compact) - printPipelineCompact(processors, buffer, settings.query_pipeline_options.header); + printPipelineCompact(processors, buf, settings.query_pipeline_options.header); else - printPipeline(processors, buffer); + printPipeline(processors, buf); } else { - plan.explainPipeline(buffer, settings.query_pipeline_options); + plan.explainPipeline(buf, settings.query_pipeline_options); } } - fillColumn(*res_columns[0], ss.str()); + fillColumn(*res_columns[0], buf.str()); return std::make_shared(sample_block.cloneWithColumns(std::move(res_columns))); } diff --git a/src/Interpreters/InterpreterShowAccessQuery.cpp b/src/Interpreters/InterpreterShowAccessQuery.cpp index 5f28c49c0bc..7bec7c411f0 100644 --- a/src/Interpreters/InterpreterShowAccessQuery.cpp +++ b/src/Interpreters/InterpreterShowAccessQuery.cpp @@ -34,13 +34,12 @@ BlockInputStreamPtr InterpreterShowAccessQuery::executeImpl() const /// Build the result column. MutableColumnPtr column = ColumnString::create(); - std::stringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString buf; for (const auto & query : queries) { - ss.str(""); - formatAST(*query, ss, false, true); - column->insert(ss.str()); + buf.restart(); + formatAST(*query, buf, false, true); + column->insert(buf.str()); } String desc = "ACCESS"; diff --git a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp index 749a5811e13..a81245adfc9 100644 --- a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp @@ -238,21 +238,19 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl() /// Build the result column. MutableColumnPtr column = ColumnString::create(); - std::stringstream create_query_ss; - create_query_ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString create_query_buf; for (const auto & create_query : create_queries) { - formatAST(*create_query, create_query_ss, false, true); - column->insert(create_query_ss.str()); - create_query_ss.str(""); + formatAST(*create_query, create_query_buf, false, true); + column->insert(create_query_buf.str()); + create_query_buf.restart(); } /// Prepare description of the result column. - std::stringstream desc_ss; - desc_ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString desc_buf; const auto & show_query = query_ptr->as(); - formatAST(show_query, desc_ss, false, true); - String desc = desc_ss.str(); + formatAST(show_query, desc_buf, false, true); + String desc = desc_buf.str(); String prefix = "SHOW "; if (startsWith(desc, prefix)) desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix. diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 8861914a68a..32f461863c9 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -78,10 +78,9 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() create.uuid = UUIDHelpers::Nil; } - std::stringstream stream; - stream.exceptions(std::ios::failbit); - formatAST(*create_query, stream, false, false); - String res = stream.str(); + WriteBufferFromOwnString buf; + formatAST(*create_query, buf, false, false); + String res = buf.str(); MutableColumnPtr column = ColumnString::create(); column->insert(res); diff --git a/src/Interpreters/InterpreterShowGrantsQuery.cpp b/src/Interpreters/InterpreterShowGrantsQuery.cpp index 7de51b6a7ee..a2ddc5eec27 100644 --- a/src/Interpreters/InterpreterShowGrantsQuery.cpp +++ b/src/Interpreters/InterpreterShowGrantsQuery.cpp @@ -118,21 +118,19 @@ BlockInputStreamPtr InterpreterShowGrantsQuery::executeImpl() /// Build the result column. MutableColumnPtr column = ColumnString::create(); - std::stringstream grant_ss; - grant_ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString grant_buf; for (const auto & grant_query : grant_queries) { - grant_ss.str(""); - formatAST(*grant_query, grant_ss, false, true); - column->insert(grant_ss.str()); + grant_buf.restart(); + formatAST(*grant_query, grant_buf, false, true); + column->insert(grant_buf.str()); } /// Prepare description of the result column. - std::stringstream desc_ss; - desc_ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString desc_buf; const auto & show_query = query_ptr->as(); - formatAST(show_query, desc_ss, false, true); - String desc = desc_ss.str(); + formatAST(show_query, desc_buf, false, true); + String desc = desc_buf.str(); String prefix = "SHOW "; if (desc.starts_with(prefix)) desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix. diff --git a/src/Interpreters/QueryAliasesVisitor.cpp b/src/Interpreters/QueryAliasesVisitor.cpp index 9de1d04990d..00c337920f4 100644 --- a/src/Interpreters/QueryAliasesVisitor.cpp +++ b/src/Interpreters/QueryAliasesVisitor.cpp @@ -20,13 +20,12 @@ namespace ErrorCodes static String wrongAliasMessage(const ASTPtr & ast, const ASTPtr & prev_ast, const String & alias) { - std::stringstream message; - message.exceptions(std::ios::failbit); - message << "Different expressions with the same alias " << backQuoteIfNeed(alias) << ":" << std::endl; + WriteBufferFromOwnString message; + message << "Different expressions with the same alias " << backQuoteIfNeed(alias) << ":\n"; formatAST(*ast, message, false, true); - message << std::endl << "and" << std::endl; + message << "\nand\n"; formatAST(*prev_ast, message, false, true); - message << std::endl; + message << '\n'; return message.str(); } diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index d74156d11d8..89130808e15 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -246,7 +247,7 @@ void ASTAlterCommand::formatImpl( << "PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); settings.ostr << (settings.hilite ? hilite_keyword : "") - << " FROM " << (settings.hilite ? hilite_none : "") << std::quoted(from, '\''); + << " FROM " << (settings.hilite ? hilite_none : "") << DB::quote << from; } else if (type == ASTAlterCommand::FREEZE_PARTITION) { @@ -256,7 +257,7 @@ void ASTAlterCommand::formatImpl( if (!with_name.empty()) { settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") - << " " << std::quoted(with_name, '\''); + << " " << DB::quote << with_name; } } else if (type == ASTAlterCommand::FREEZE_ALL) @@ -266,7 +267,7 @@ void ASTAlterCommand::formatImpl( if (!with_name.empty()) { settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") - << " " << std::quoted(with_name, '\''); + << " " << DB::quote << with_name; } } else if (type == ASTAlterCommand::DELETE) diff --git a/src/Parsers/ASTAsterisk.cpp b/src/Parsers/ASTAsterisk.cpp index 95a63586685..ed733e62ca3 100644 --- a/src/Parsers/ASTAsterisk.cpp +++ b/src/Parsers/ASTAsterisk.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 27ece3e18c2..4c14230e926 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Parsers/ASTColumnsMatcher.cpp b/src/Parsers/ASTColumnsMatcher.cpp index aab8e841981..45799cb7ffe 100644 --- a/src/Parsers/ASTColumnsMatcher.cpp +++ b/src/Parsers/ASTColumnsMatcher.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index a8f39079902..7c0fd0eb734 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTConstraintDeclaration.cpp b/src/Parsers/ASTConstraintDeclaration.cpp index 371bfa40f54..7d74837478c 100644 --- a/src/Parsers/ASTConstraintDeclaration.cpp +++ b/src/Parsers/ASTConstraintDeclaration.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 4efe5476395..a193433c988 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTCreateQuotaQuery.cpp b/src/Parsers/ASTCreateQuotaQuery.cpp index 88516fb6eac..7e570b889e3 100644 --- a/src/Parsers/ASTCreateQuotaQuery.cpp +++ b/src/Parsers/ASTCreateQuotaQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTCreateRoleQuery.cpp b/src/Parsers/ASTCreateRoleQuery.cpp index 5ccfd9c6bd5..73b523a5bfe 100644 --- a/src/Parsers/ASTCreateRoleQuery.cpp +++ b/src/Parsers/ASTCreateRoleQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTCreateRowPolicyQuery.cpp b/src/Parsers/ASTCreateRowPolicyQuery.cpp index 6224b534851..241d3ff051a 100644 --- a/src/Parsers/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/ASTCreateRowPolicyQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -62,14 +63,13 @@ namespace void formatForClauses(const std::vector> & conditions, bool alter, const IAST::FormatSettings & settings) { std::vector> conditions_as_strings; - std::stringstream temp_sstream; - temp_sstream.exceptions(std::ios::failbit); - IAST::FormatSettings temp_settings(temp_sstream, settings); + WriteBufferFromOwnString temp_buf; + IAST::FormatSettings temp_settings(temp_buf, settings); for (const auto & [condition_type, condition] : conditions) { formatConditionalExpression(condition, temp_settings); - conditions_as_strings.emplace_back(condition_type, temp_sstream.str()); - temp_sstream.str(""); + conditions_as_strings.emplace_back(condition_type, temp_buf.str()); + temp_buf.restart(); } boost::container::flat_set commands; diff --git a/src/Parsers/ASTCreateSettingsProfileQuery.cpp b/src/Parsers/ASTCreateSettingsProfileQuery.cpp index 77c2f1b22d7..84f8309462e 100644 --- a/src/Parsers/ASTCreateSettingsProfileQuery.cpp +++ b/src/Parsers/ASTCreateSettingsProfileQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTCreateUserQuery.cpp b/src/Parsers/ASTCreateUserQuery.cpp index 0ccc2232734..4b2aa70785a 100644 --- a/src/Parsers/ASTCreateUserQuery.cpp +++ b/src/Parsers/ASTCreateUserQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTDictionary.cpp b/src/Parsers/ASTDictionary.cpp index e0785f3bd49..878f6000aa9 100644 --- a/src/Parsers/ASTDictionary.cpp +++ b/src/Parsers/ASTDictionary.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { diff --git a/src/Parsers/ASTDictionaryAttributeDeclaration.cpp b/src/Parsers/ASTDictionaryAttributeDeclaration.cpp index 05ba48ace7b..e9c50839a98 100644 --- a/src/Parsers/ASTDictionaryAttributeDeclaration.cpp +++ b/src/Parsers/ASTDictionaryAttributeDeclaration.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Parsers/ASTDropAccessEntityQuery.cpp b/src/Parsers/ASTDropAccessEntityQuery.cpp index fe98d8b4158..1df176c24ec 100644 --- a/src/Parsers/ASTDropAccessEntityQuery.cpp +++ b/src/Parsers/ASTDropAccessEntityQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTDropQuery.cpp b/src/Parsers/ASTDropQuery.cpp index 0ced4a8ea96..b09b588ca6e 100644 --- a/src/Parsers/ASTDropQuery.cpp +++ b/src/Parsers/ASTDropQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Parsers/ASTExpressionList.cpp b/src/Parsers/ASTExpressionList.cpp index de38e1fd7ea..2724465537f 100644 --- a/src/Parsers/ASTExpressionList.cpp +++ b/src/Parsers/ASTExpressionList.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 66565eeaf8f..76a52b8c641 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTFunctionWithKeyValueArguments.cpp b/src/Parsers/ASTFunctionWithKeyValueArguments.cpp index 5f1e78b61da..d94490ab8b3 100644 --- a/src/Parsers/ASTFunctionWithKeyValueArguments.cpp +++ b/src/Parsers/ASTFunctionWithKeyValueArguments.cpp @@ -2,6 +2,7 @@ #include #include +#include namespace DB { diff --git a/src/Parsers/ASTGrantQuery.cpp b/src/Parsers/ASTGrantQuery.cpp index fefebc4ae4a..2610836c759 100644 --- a/src/Parsers/ASTGrantQuery.cpp +++ b/src/Parsers/ASTGrantQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTIdentifier.cpp b/src/Parsers/ASTIdentifier.cpp index 5a66bc7891d..d51f37a0047 100644 --- a/src/Parsers/ASTIdentifier.cpp +++ b/src/Parsers/ASTIdentifier.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTIndexDeclaration.cpp b/src/Parsers/ASTIndexDeclaration.cpp index e89f9bf26ed..0e8f0d0f7e8 100644 --- a/src/Parsers/ASTIndexDeclaration.cpp +++ b/src/Parsers/ASTIndexDeclaration.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Parsers/ASTInsertQuery.cpp b/src/Parsers/ASTInsertQuery.cpp index dc9b9f092ac..4096c484059 100644 --- a/src/Parsers/ASTInsertQuery.cpp +++ b/src/Parsers/ASTInsertQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTKillQueryQuery.cpp b/src/Parsers/ASTKillQueryQuery.cpp index 293b95b93bf..72bdd7d6b0b 100644 --- a/src/Parsers/ASTKillQueryQuery.cpp +++ b/src/Parsers/ASTKillQueryQuery.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { diff --git a/src/Parsers/ASTLiteral.cpp b/src/Parsers/ASTLiteral.cpp index cd9c389f336..ed6790499fb 100644 --- a/src/Parsers/ASTLiteral.cpp +++ b/src/Parsers/ASTLiteral.cpp @@ -73,4 +73,9 @@ void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const } } +void ASTLiteral::formatImplWithoutAlias(const FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << applyVisitor(FieldVisitorToString(), value); +} + } diff --git a/src/Parsers/ASTLiteral.h b/src/Parsers/ASTLiteral.h index 18f440a81a4..672bc6ddc3e 100644 --- a/src/Parsers/ASTLiteral.h +++ b/src/Parsers/ASTLiteral.h @@ -43,10 +43,7 @@ public: void updateTreeHashImpl(SipHash & hash_state) const override; protected: - void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override - { - settings.ostr << applyVisitor(FieldVisitorToString(), value); - } + void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; }; diff --git a/src/Parsers/ASTNameTypePair.cpp b/src/Parsers/ASTNameTypePair.cpp index 80b0f7f8ec6..e4066081a9b 100644 --- a/src/Parsers/ASTNameTypePair.cpp +++ b/src/Parsers/ASTNameTypePair.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Parsers/ASTOptimizeQuery.cpp b/src/Parsers/ASTOptimizeQuery.cpp index 92968f2b277..ae83952899d 100644 --- a/src/Parsers/ASTOptimizeQuery.cpp +++ b/src/Parsers/ASTOptimizeQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { diff --git a/src/Parsers/ASTOrderByElement.cpp b/src/Parsers/ASTOrderByElement.cpp index 48290b2669a..884d69a18e3 100644 --- a/src/Parsers/ASTOrderByElement.cpp +++ b/src/Parsers/ASTOrderByElement.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTPartition.cpp b/src/Parsers/ASTPartition.cpp index d24575b7f43..06bfe4f5217 100644 --- a/src/Parsers/ASTPartition.cpp +++ b/src/Parsers/ASTPartition.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { diff --git a/src/Parsers/ASTQualifiedAsterisk.cpp b/src/Parsers/ASTQualifiedAsterisk.cpp index 0cda01cecac..2491dcb36b7 100644 --- a/src/Parsers/ASTQualifiedAsterisk.cpp +++ b/src/Parsers/ASTQualifiedAsterisk.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { diff --git a/src/Parsers/ASTQueryParameter.cpp b/src/Parsers/ASTQueryParameter.cpp index 915ecd5e7e4..c10cced23ce 100644 --- a/src/Parsers/ASTQueryParameter.cpp +++ b/src/Parsers/ASTQueryParameter.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTQueryWithOnCluster.cpp b/src/Parsers/ASTQueryWithOnCluster.cpp index cb77d351d54..60e96b1dbe1 100644 --- a/src/Parsers/ASTQueryWithOnCluster.cpp +++ b/src/Parsers/ASTQueryWithOnCluster.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTQueryWithOutput.h b/src/Parsers/ASTQueryWithOutput.h index 9018d5661d9..92f9331f259 100644 --- a/src/Parsers/ASTQueryWithOutput.h +++ b/src/Parsers/ASTQueryWithOutput.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB diff --git a/src/Parsers/ASTQueryWithTableAndOutput.cpp b/src/Parsers/ASTQueryWithTableAndOutput.cpp index 3a776590f80..d44ba988d7a 100644 --- a/src/Parsers/ASTQueryWithTableAndOutput.cpp +++ b/src/Parsers/ASTQueryWithTableAndOutput.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Parsers/ASTRenameQuery.h b/src/Parsers/ASTRenameQuery.h index 951abbe4419..8d300f430b3 100644 --- a/src/Parsers/ASTRenameQuery.h +++ b/src/Parsers/ASTRenameQuery.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTRolesOrUsersSet.cpp b/src/Parsers/ASTRolesOrUsersSet.cpp index a666d8ae1d5..1e7cd79f527 100644 --- a/src/Parsers/ASTRolesOrUsersSet.cpp +++ b/src/Parsers/ASTRolesOrUsersSet.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Parsers/ASTRowPolicyName.cpp b/src/Parsers/ASTRowPolicyName.cpp index 5e3c494ccd3..3d1ac5621db 100644 --- a/src/Parsers/ASTRowPolicyName.cpp +++ b/src/Parsers/ASTRowPolicyName.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB diff --git a/src/Parsers/ASTSampleRatio.cpp b/src/Parsers/ASTSampleRatio.cpp index 8c5901d121d..03a4c9adf23 100644 --- a/src/Parsers/ASTSampleRatio.cpp +++ b/src/Parsers/ASTSampleRatio.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -34,5 +35,9 @@ String ASTSampleRatio::toString(Rational ratio) return toString(ratio.numerator) + " / " + toString(ratio.denominator); } +void ASTSampleRatio::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << toString(ratio); +} } diff --git a/src/Parsers/ASTSampleRatio.h b/src/Parsers/ASTSampleRatio.h index 787833eb4f3..e8953dec022 100644 --- a/src/Parsers/ASTSampleRatio.h +++ b/src/Parsers/ASTSampleRatio.h @@ -35,10 +35,7 @@ public: static String toString(BigNum num); static String toString(Rational ratio); - void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override - { - settings.ostr << toString(ratio); - } + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; }; } diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index 499761c4634..915d1f71925 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTSelectWithUnionQuery.cpp b/src/Parsers/ASTSelectWithUnionQuery.cpp index 96cac839c58..9c13a1a4ff3 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.cpp +++ b/src/Parsers/ASTSelectWithUnionQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index 8835d1dc7da..c8a2b3b37e8 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTSetRoleQuery.cpp b/src/Parsers/ASTSetRoleQuery.cpp index b5e0c05e083..e59e103b774 100644 --- a/src/Parsers/ASTSetRoleQuery.cpp +++ b/src/Parsers/ASTSetRoleQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTSettingsProfileElement.cpp b/src/Parsers/ASTSettingsProfileElement.cpp index c0fb2965a2d..2422126219f 100644 --- a/src/Parsers/ASTSettingsProfileElement.cpp +++ b/src/Parsers/ASTSettingsProfileElement.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTShowAccessEntitiesQuery.cpp b/src/Parsers/ASTShowAccessEntitiesQuery.cpp index e87baebba33..bacde098640 100644 --- a/src/Parsers/ASTShowAccessEntitiesQuery.cpp +++ b/src/Parsers/ASTShowAccessEntitiesQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Parsers/ASTShowCreateAccessEntityQuery.cpp b/src/Parsers/ASTShowCreateAccessEntityQuery.cpp index bc309ab5c44..f870c98071c 100644 --- a/src/Parsers/ASTShowCreateAccessEntityQuery.cpp +++ b/src/Parsers/ASTShowCreateAccessEntityQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTShowGrantsQuery.cpp b/src/Parsers/ASTShowGrantsQuery.cpp index 26ae506d7d4..4011cfc522c 100644 --- a/src/Parsers/ASTShowGrantsQuery.cpp +++ b/src/Parsers/ASTShowGrantsQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTShowPrivilegesQuery.cpp b/src/Parsers/ASTShowPrivilegesQuery.cpp deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/src/Parsers/ASTShowTablesQuery.cpp b/src/Parsers/ASTShowTablesQuery.cpp index 1e8dad13ad3..cd83bae06d9 100644 --- a/src/Parsers/ASTShowTablesQuery.cpp +++ b/src/Parsers/ASTShowTablesQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -21,7 +22,7 @@ void ASTShowTablesQuery::formatLike(const FormatSettings & settings) const << (not_like ? " NOT" : "") << (case_insensitive_like ? " ILIKE " : " LIKE ") << (settings.hilite ? hilite_none : "") - << std::quoted(like, '\''); + << DB::quote << like; } void ASTShowTablesQuery::formatLimit(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const diff --git a/src/Parsers/ASTSubquery.cpp b/src/Parsers/ASTSubquery.cpp index 55ea89e3f07..bfe413908b9 100644 --- a/src/Parsers/ASTSubquery.cpp +++ b/src/Parsers/ASTSubquery.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 9cbb6ae94f6..4ed0ecd3a91 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index f37631769b8..39283a3168e 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTTablesInSelectQuery.cpp b/src/Parsers/ASTTablesInSelectQuery.cpp index eb3446ca1c4..8d131a848f7 100644 --- a/src/Parsers/ASTTablesInSelectQuery.cpp +++ b/src/Parsers/ASTTablesInSelectQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTUseQuery.h b/src/Parsers/ASTUseQuery.h index 2127bf9f2c0..4e4a13c2a7f 100644 --- a/src/Parsers/ASTUseQuery.h +++ b/src/Parsers/ASTUseQuery.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB diff --git a/src/Parsers/ASTUserNameWithHost.cpp b/src/Parsers/ASTUserNameWithHost.cpp index 13d34b99b3d..b99ea5ab8d4 100644 --- a/src/Parsers/ASTUserNameWithHost.cpp +++ b/src/Parsers/ASTUserNameWithHost.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Parsers/ASTWithAlias.cpp b/src/Parsers/ASTWithAlias.cpp index ad93102e1b7..1feb89f4bdc 100644 --- a/src/Parsers/ASTWithAlias.cpp +++ b/src/Parsers/ASTWithAlias.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB diff --git a/src/Parsers/ASTWithElement.cpp b/src/Parsers/ASTWithElement.cpp index 9d22286c2fd..00a82b703af 100644 --- a/src/Parsers/ASTWithElement.cpp +++ b/src/Parsers/ASTWithElement.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { diff --git a/src/Parsers/CommonParsers.cpp b/src/Parsers/CommonParsers.cpp index 47868f5df48..d7a9ed60ac3 100644 --- a/src/Parsers/CommonParsers.cpp +++ b/src/Parsers/CommonParsers.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include /// strncmp, strncasecmp diff --git a/src/Parsers/DumpASTNode.h b/src/Parsers/DumpASTNode.h index 01447850c74..1208aeca2a9 100644 --- a/src/Parsers/DumpASTNode.h +++ b/src/Parsers/DumpASTNode.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -14,7 +15,7 @@ namespace DB class DumpASTNode { public: - DumpASTNode(const IAST & ast_, std::ostream * ostr_, size_t & depth, const char * label_ = nullptr) + DumpASTNode(const IAST & ast_, WriteBuffer * ostr_, size_t & depth, const char * label_ = nullptr) : ast(ast_), ostr(ostr_), indent(depth), @@ -24,12 +25,12 @@ public: if (!ostr) return; if (label && visit_depth == 0) - (*ostr) << "-- " << label << std::endl; + (*ostr) << "-- " << label << '\n'; ++visit_depth; (*ostr) << String(indent, ' '); printNode(); - (*ostr) << std::endl; + (*ostr) << '\n'; } ~DumpASTNode() @@ -38,7 +39,7 @@ public: return; --visit_depth; if (label && visit_depth == 0) - (*ostr) << "--" << std::endl; + (*ostr) << "--\n"; } template @@ -50,14 +51,14 @@ public: (*ostr) << (str_indent ? String(str_indent) : String(indent, ' ')); (*ostr) << '(' << name << ' ' << value << ')'; if (!str_indent) - (*ostr) << std::endl; + (*ostr) << '\n'; } size_t & getDepth() { return visit_depth; } private: const IAST & ast; - std::ostream * ostr; + WriteBuffer * ostr; size_t indent; size_t & visit_depth; /// shared with children const char * label; @@ -77,7 +78,7 @@ private: } }; -inline void dumpAST(const IAST & ast, std::ostream & ostr, DumpASTNode * parent = nullptr) +inline void dumpAST(const IAST & ast, WriteBuffer & ostr, DumpASTNode * parent = nullptr) { size_t depth = 0; DumpASTNode dump(ast, &ostr, (parent ? parent->getDepth() : depth)); @@ -95,7 +96,6 @@ public: DebugASTLog() : log(nullptr) { - ss.exceptions(std::ios::failbit); if constexpr (_enable) log = &Poco::Logger::get("AST"); } @@ -103,14 +103,14 @@ public: ~DebugASTLog() { if constexpr (_enable) - LOG_DEBUG(log, ss.str()); + LOG_DEBUG(log, buf.str()); } - std::ostream * stream() { return (_enable ? &ss : nullptr); } + WriteBuffer * stream() { return (_enable ? &buf : nullptr); } private: Poco::Logger * log; - std::stringstream ss; + WriteBufferFromOwnString buf; }; diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index d716a796b77..4b7d3c2ea40 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -89,10 +89,9 @@ size_t IAST::checkDepthImpl(size_t max_depth, size_t level) const std::string IAST::formatForErrorMessage() const { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - format(FormatSettings(ss, true /* one line */)); - return ss.str(); + WriteBufferFromOwnString buf; + format(FormatSettings(buf, true /* one line */)); + return buf.str(); } void IAST::cloneChildren() @@ -112,8 +111,6 @@ String IAST::getColumnName() const void IAST::FormatSettings::writeIdentifier(const String & name) const { - WriteBufferFromOStream out(ostr, 32); - switch (identifier_quoting_style) { case IdentifierQuotingStyle::None: @@ -121,36 +118,34 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const if (always_quote_identifiers) throw Exception("Incompatible arguments: always_quote_identifiers = true && identifier_quoting_style == IdentifierQuotingStyle::None", ErrorCodes::BAD_ARGUMENTS); - writeString(name, out); + writeString(name, ostr); break; } case IdentifierQuotingStyle::Backticks: { if (always_quote_identifiers) - writeBackQuotedString(name, out); + writeBackQuotedString(name, ostr); else - writeProbablyBackQuotedString(name, out); + writeProbablyBackQuotedString(name, ostr); break; } case IdentifierQuotingStyle::DoubleQuotes: { if (always_quote_identifiers) - writeDoubleQuotedString(name, out); + writeDoubleQuotedString(name, ostr); else - writeProbablyDoubleQuotedString(name, out); + writeProbablyDoubleQuotedString(name, ostr); break; } case IdentifierQuotingStyle::BackticksMySQL: { if (always_quote_identifiers) - writeBackQuotedStringMySQL(name, out); + writeBackQuotedStringMySQL(name, ostr); else - writeProbablyBackQuotedStringMySQL(name, out); + writeProbablyBackQuotedStringMySQL(name, ostr); break; } } - - out.next(); } } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index cc9e593d7cb..d9fd71378f7 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -161,7 +162,7 @@ public: /// Format settings. struct FormatSettings { - std::ostream & ostr; + WriteBuffer & ostr; bool hilite = false; bool one_line; bool always_quote_identifiers = false; @@ -169,13 +170,13 @@ public: char nl_or_ws; - FormatSettings(std::ostream & ostr_, bool one_line_) + FormatSettings(WriteBuffer & ostr_, bool one_line_) : ostr(ostr_), one_line(one_line_) { nl_or_ws = one_line ? ' ' : '\n'; } - FormatSettings(std::ostream & ostr_, const FormatSettings & other) + FormatSettings(WriteBuffer & ostr_, const FormatSettings & other) : ostr(ostr_), hilite(other.hilite), one_line(other.one_line), always_quote_identifiers(other.always_quote_identifiers), identifier_quoting_style(other.identifier_quoting_style) { @@ -242,17 +243,17 @@ private: template std::string IAST::formatForErrorMessage(const AstArray & array) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString buf; for (size_t i = 0; i < array.size(); ++i) { if (i > 0) { - ss << ", "; + const char * delim = ", "; + buf.write(delim, strlen(delim)); } - array[i]->format(IAST::FormatSettings(ss, true /* one line */)); + array[i]->format(IAST::FormatSettings(buf, true /* one line */)); } - return ss.str(); + return buf.str(); } } diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index e19dc715d51..3a258df099e 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -5,9 +5,9 @@ namespace DB { -void formatAST(const IAST & ast, std::ostream & s, bool hilite, bool one_line) +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line) { - IAST::FormatSettings settings(s, one_line); + IAST::FormatSettings settings(buf, one_line); settings.hilite = hilite; ast.format(settings); @@ -15,10 +15,9 @@ void formatAST(const IAST & ast, std::ostream & s, bool hilite, bool one_line) String serializeAST(const IAST & ast, bool one_line) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - formatAST(ast, ss, false, one_line); - return ss.str(); + WriteBufferFromOwnString buf; + formatAST(ast, buf, false, one_line); + return buf.str(); } } diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index 685c504514e..bee89521812 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -7,23 +7,25 @@ namespace DB { +class WriteBuffer; + /** Takes a syntax tree and turns it back into text. * In case of INSERT query, the data will be missing. */ -void formatAST(const IAST & ast, std::ostream & s, bool hilite = true, bool one_line = false); +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false); String serializeAST(const IAST & ast, bool one_line = true); -inline std::ostream & operator<<(std::ostream & os, const IAST & ast) -{ - formatAST(ast, os, false, true); - return os; -} - -inline std::ostream & operator<<(std::ostream & os, const ASTPtr & ast) -{ - formatAST(*ast, os, false, true); - return os; -} +//inline std::ostream & operator<<(std::ostream & os, const IAST & ast) +//{ +// formatAST(ast, os, false, true); +// return os; +//} +// +//inline std::ostream & operator<<(std::ostream & os, const ASTPtr & ast) +//{ +// formatAST(*ast, os, false, true); +// return os; +//} } diff --git a/src/Parsers/formatSettingName.cpp b/src/Parsers/formatSettingName.cpp index c305496fdb3..351a1eb8ade 100644 --- a/src/Parsers/formatSettingName.cpp +++ b/src/Parsers/formatSettingName.cpp @@ -2,13 +2,13 @@ #include #include #include -#include +#include namespace DB { -void formatSettingName(const String & setting_name, std::ostream & out) +void formatSettingName(const String & setting_name, WriteBuffer & out) { if (isValidIdentifier(setting_name)) { diff --git a/src/Parsers/formatSettingName.h b/src/Parsers/formatSettingName.h index a700d347a5f..ba819ee2b4c 100644 --- a/src/Parsers/formatSettingName.h +++ b/src/Parsers/formatSettingName.h @@ -7,9 +7,11 @@ namespace DB { +class WriteBuffer; + /// Outputs built-in or custom setting's name. /// The function is like backQuoteIfNeed() but didn't quote with backticks /// if the name consists of identifiers joined with dots. -void formatSettingName(const String & setting_name, std::ostream & out); +void formatSettingName(const String & setting_name, WriteBuffer & out); } diff --git a/src/Parsers/queryToString.cpp b/src/Parsers/queryToString.cpp index 44ea721485f..9721aa1f128 100644 --- a/src/Parsers/queryToString.cpp +++ b/src/Parsers/queryToString.cpp @@ -1,6 +1,5 @@ #include #include -#include namespace DB { @@ -11,9 +10,6 @@ namespace DB String queryToString(const IAST & query) { - std::ostringstream out; - out.exceptions(std::ios::failbit); - formatAST(query, out, false, true); - return out.str(); + return serializeAST(query); } } diff --git a/src/Parsers/tests/create_parser.cpp b/src/Parsers/tests/create_parser.cpp index fbdc967fa2a..c241b353b4f 100644 --- a/src/Parsers/tests/create_parser.cpp +++ b/src/Parsers/tests/create_parser.cpp @@ -4,6 +4,7 @@ #include #include #include +#include int main(int, char **) @@ -14,7 +15,8 @@ int main(int, char **) ParserCreateQuery parser; ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); - formatAST(*ast, std::cerr); + WriteBufferFromOStream out(std::cerr, 4096); + formatAST(*ast, out); std::cerr << std::endl; return 0; diff --git a/src/Parsers/tests/gtest_dictionary_parser.cpp b/src/Parsers/tests/gtest_dictionary_parser.cpp index c418759aa21..b051cedfb23 100644 --- a/src/Parsers/tests/gtest_dictionary_parser.cpp +++ b/src/Parsers/tests/gtest_dictionary_parser.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -17,10 +18,9 @@ using namespace DB; static String astToString(IAST * ast) { - std::ostringstream oss; - oss.exceptions(std::ios::failbit); - dumpAST(*ast, oss); - return oss.str(); + WriteBufferFromOwnString buf; + dumpAST(*ast, buf); + return buf.str(); } /// Tests for external dictionaries DDL parser diff --git a/src/Parsers/tests/select_parser.cpp b/src/Parsers/tests/select_parser.cpp index 7711f0d2527..7c18563659d 100644 --- a/src/Parsers/tests/select_parser.cpp +++ b/src/Parsers/tests/select_parser.cpp @@ -3,6 +3,7 @@ #include #include #include +#include int main(int, char **) @@ -25,7 +26,8 @@ try ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); std::cout << "Success." << std::endl; - formatAST(*ast, std::cerr); + WriteBufferFromOStream out(std::cerr, 4096); + formatAST(*ast, out); std::cout << std::endl; return 0; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 597ff5e8fee..aea740a3a26 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1693,13 +1693,12 @@ std::vector ReplicatedMergeTreeQueue::getMutationsStatu for (const MutationCommand & command : entry.commands) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - formatAST(*command.ast, ss, false, true); + WriteBufferFromOwnString buf; + formatAST(*command.ast, buf, false, true); result.push_back(MergeTreeMutationStatus { entry.znode_name, - ss.str(), + buf.str(), entry.create_time, entry.block_numbers, parts_to_mutate, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 48f05b50675..d06706f9109 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -18,10 +18,9 @@ static String formattedAST(const ASTPtr & ast) { if (!ast) return ""; - std::stringstream ss; - ss.exceptions(std::ios::failbit); - formatAST(*ast, ss, false, true); - return ss.str(); + WriteBufferFromOwnString buf; + formatAST(*ast, buf, false, true); + return buf.str(); } ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data, const StorageMetadataPtr & metadata_snapshot) diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 53c9b50cb9d..c57ea0f1d77 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -126,10 +126,9 @@ std::shared_ptr MutationCommands::ast() const void MutationCommands::writeText(WriteBuffer & out) const { - std::stringstream commands_ss; - commands_ss.exceptions(std::ios::failbit); - formatAST(*ast(), commands_ss, /* hilite = */ false, /* one_line = */ true); - out << escape << commands_ss.str(); + WriteBufferFromOwnString commands_buf; + formatAST(*ast(), commands_buf, /* hilite = */ false, /* one_line = */ true); + out << escape << commands_buf.str(); } void MutationCommands::readText(ReadBuffer & in) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4ec6d748738..fbeb188d649 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -538,13 +538,12 @@ std::vector StorageMergeTree::getMutationsStatus() cons for (const MutationCommand & command : entry.commands) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - formatAST(*command.ast, ss, false, true); + WriteBufferFromOwnString buf; + formatAST(*command.ast, buf, false, true); result.push_back(MergeTreeMutationStatus { entry.file_name, - ss.str(), + buf.str(), entry.create_time, block_numbers_map, parts_to_do_names, diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 3148ab1112a..81d3303e262 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -220,8 +220,7 @@ String transformQueryForExternalDatabase( ASTPtr select_ptr = select; dropAliases(select_ptr); - std::stringstream out; - out.exceptions(std::ios::failbit); + WriteBufferFromOwnString out; IAST::FormatSettings settings(out, true); settings.identifier_quoting_style = identifier_quoting_style; settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None; diff --git a/utils/db-generator/query_db_generator.cpp b/utils/db-generator/query_db_generator.cpp index c8aae4a56f3..e3c5b862dbc 100644 --- a/utils/db-generator/query_db_generator.cpp +++ b/utils/db-generator/query_db_generator.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -828,9 +829,9 @@ FuncRet arithmeticFunc(DB::ASTPtr ch, std::map & columns) FuncRet r(ret_type, ""); if (no_indent) { - std::ostringstream ss; - formatAST(*ch, ss); - r.value = ss.str(); + DB::WriteBufferFromOwnString buf; + formatAST(*ch, buf); + r.value = buf.str(); } return r; } @@ -990,10 +991,10 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) { if (no_indent) { - std::ostringstream ss; - formatAST(*ch, ss); + DB::WriteBufferFromOwnString buf; + formatAST(*ch, buf); auto r = func_to_return_type[boost::algorithm::to_lower_copy(x->name)]; - r.value = ss.str(); + r.value = buf.str(); return r; } return func_to_return_type[boost::algorithm::to_lower_copy(x->name)]; @@ -1003,11 +1004,11 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) { if (no_indent) { - std::ostringstream ss; - formatAST(*ch, ss); + DB::WriteBufferFromOwnString buf; + formatAST(*ch, buf); return FuncRet( func_to_param_type[boost::algorithm::to_lower_copy(x->name)], - ss.str()); + buf.str()); } return FuncRet( func_to_param_type[boost::algorithm::to_lower_copy(x->name)], From 3c6794bf37bc126751f67aeca11c97df75c7b707 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Mon, 9 Nov 2020 20:22:05 +0300 Subject: [PATCH 47/94] 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 48/94] Retry CI From 5cdfcfb3077884357b8eed9b8a11b856b81b54e7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 9 Nov 2020 22:07:38 +0300 Subject: [PATCH 49/94] remove other stringstreams --- base/mysqlxx/Query.h | 2 +- programs/client/Client.cpp | 7 ++-- programs/format/Format.cpp | 1 + src/Access/DiskAccessStorage.cpp | 2 +- src/Access/LDAPAccessStorage.cpp | 2 +- src/Access/UsersConfigAccessStorage.cpp | 2 +- src/Common/HTMLForm.h | 4 +- src/Common/ThreadProfileEvents.cpp | 3 +- src/Common/UInt128.h | 8 ++-- src/Common/parseGlobs.cpp | 33 +++++++-------- .../gtest_getMultipleValuesFromConfig.cpp | 3 +- .../tests/gtest_sensitive_data_masker.cpp | 12 ++++-- .../tests/gtest_data_type_get_common_type.cpp | 2 +- src/Dictionaries/DictionaryStructure.cpp | 4 +- .../tests/gtest_dictionary_configuration.cpp | 2 +- src/IO/WriteHelpers.cpp | 8 ++++ src/IO/WriteHelpers.h | 2 + src/Parsers/IAST.cpp | 10 +++++ src/Parsers/IAST.h | 8 +--- .../MySQL/tests/gtest_create_parser.cpp | 4 +- src/Parsers/iostream_debug_helpers.cpp | 9 +++-- src/Parsers/ya.make | 1 - .../Formats/Impl/AvroRowInputFormat.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 40 +++++++++---------- src/Storages/MergeTree/MergeTreeData.cpp | 40 +++++++++---------- src/Storages/StorageDistributed.cpp | 16 ++++---- src/Storages/StorageS3.cpp | 16 +++----- src/Storages/System/StorageSystemUsers.cpp | 2 +- src/Storages/tests/gtest_storage_log.cpp | 6 +-- utils/check-style/check-style | 2 +- 30 files changed, 136 insertions(+), 117 deletions(-) diff --git a/base/mysqlxx/Query.h b/base/mysqlxx/Query.h index 1d3ab9678d5..d0a905e2031 100644 --- a/base/mysqlxx/Query.h +++ b/base/mysqlxx/Query.h @@ -77,7 +77,7 @@ public: private: Connection * conn; - std::ostringstream query_buf; + std::ostringstream query_buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM void executeImpl(); }; diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index f9e4b357aa9..98c1463ab23 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1159,13 +1159,13 @@ private: ASTPtr ast_to_process; try { - std::stringstream dump_before_fuzz; + WriteBufferFromOwnString dump_before_fuzz; fuzz_base->dumpTree(dump_before_fuzz); auto base_before_fuzz = fuzz_base->formatForErrorMessage(); ast_to_process = fuzz_base->clone(); - std::stringstream dump_of_cloned_ast; + WriteBufferFromOwnString dump_of_cloned_ast; ast_to_process->dumpTree(dump_of_cloned_ast); // Run the original query as well. @@ -1187,7 +1187,8 @@ private: fprintf(stderr, "dump of cloned ast:\n%s\n", dump_of_cloned_ast.str().c_str()); fprintf(stderr, "dump after fuzz:\n"); - fuzz_base->dumpTree(std::cerr); + WriteBufferFromOStream cerr_buf(std::cerr, 4096); + fuzz_base->dumpTree(cerr_buf); fmt::print(stderr, "IAST::clone() is broken for some AST node. This is a bug. The original AST ('dump before fuzz') and its cloned copy ('dump of cloned AST') refer to the same nodes, which must never happen. This means that their parent node doesn't implement clone() correctly."); diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index c9981e706b0..986dc67a798 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -132,6 +132,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv) { WriteBufferFromOStream res_buf(std::cout, 4096); formatAST(*res, res_buf, hilite, oneline); + res_buf.next(); if (multiple) std::cout << "\n;\n"; std::cout << std::endl; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index b03a2791560..426c27ea799 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -355,7 +355,7 @@ String DiskAccessStorage::getStorageParamsJSON() const json.set("path", directory_path); if (readonly) json.set("readonly", readonly.load()); - std::ostringstream oss; + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(json, oss); return oss.str(); diff --git a/src/Access/LDAPAccessStorage.cpp b/src/Access/LDAPAccessStorage.cpp index c9f00f2f4ab..92de7fce8d7 100644 --- a/src/Access/LDAPAccessStorage.cpp +++ b/src/Access/LDAPAccessStorage.cpp @@ -150,7 +150,7 @@ String LDAPAccessStorage::getStorageParamsJSON() const params_json.set("server", ldap_server); params_json.set("roles", default_role_names); - std::ostringstream oss; + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(params_json, oss); diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index ba3c2d5b8e5..eb993d696c6 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -460,7 +460,7 @@ String UsersConfigAccessStorage::getStorageParamsJSON() const Poco::JSON::Object json; if (!path.empty()) json.set("path", path); - std::ostringstream oss; + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(json, oss); return oss.str(); diff --git a/src/Common/HTMLForm.h b/src/Common/HTMLForm.h index 2490d613160..2b62167dce7 100644 --- a/src/Common/HTMLForm.h +++ b/src/Common/HTMLForm.h @@ -16,13 +16,13 @@ struct HTMLForm : public Poco::Net::HTMLForm HTMLForm(const Poco::Net::HTTPRequest & request) { Poco::URI uri(request.getURI()); - std::istringstream istr(uri.getRawQuery()); + std::istringstream istr(uri.getRawQuery()); // STYLE_CHECK_ALLOW_STD_STRING_STREAM readUrl(istr); } HTMLForm(const Poco::URI & uri) { - std::istringstream istr(uri.getRawQuery()); + std::istringstream istr(uri.getRawQuery()); // STYLE_CHECK_ALLOW_STD_STRING_STREAM readUrl(istr); } diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 7b94ca0f2b2..c8a4555dabf 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -413,7 +413,8 @@ std::vector PerfEventsCounters::eventIndicesFromString(const std::string return result; } - std::istringstream iss(events_list); + + std::istringstream iss(events_list); // STYLE_CHECK_ALLOW_STD_STRING_STREAM std::string event_name; while (std::getline(iss, event_name, ',')) { diff --git a/src/Common/UInt128.h b/src/Common/UInt128.h index 7b6f8e7c7be..735b287f90f 100644 --- a/src/Common/UInt128.h +++ b/src/Common/UInt128.h @@ -6,6 +6,7 @@ #include #include +#include #ifdef __SSE4_2__ #include @@ -48,10 +49,9 @@ struct UInt128 String toHexString() const { - std::ostringstream os; - os.exceptions(std::ios::failbit); - os << std::setw(16) << std::setfill('0') << std::hex << high << low; - return String(os.str()); + String res(2 * sizeof(UInt128), 0); + writeHexUIntLowercase(*this, res.data()); + return res; } bool inline operator== (const UInt128 rhs) const { return tuple() == rhs.tuple(); } diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index f04ad1ea8a0..75539512b6d 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -1,4 +1,7 @@ #include +#include +#include +#include #include #include #include @@ -18,21 +21,21 @@ namespace DB */ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_globs) { - std::ostringstream oss_for_escaping; - oss_for_escaping.exceptions(std::ios::failbit); + /// FIXME make it better + WriteBufferFromOwnString buf_for_escaping; /// Escaping only characters that not used in glob syntax for (const auto & letter : initial_str_with_globs) { - if ((letter == '[') || (letter == ']') || (letter == '|') || (letter == '+') || (letter == '-') || (letter == '(') || (letter == ')')) - oss_for_escaping << '\\'; - oss_for_escaping << letter; + if ((letter == '[') || (letter == ']') || (letter == '|') || (letter == '+') || (letter == '-') || (letter == '(') || (letter == ')') || (letter == '\\')) + buf_for_escaping << '\\'; + buf_for_escaping << letter; } - std::string escaped_with_globs = oss_for_escaping.str(); + std::string escaped_with_globs = buf_for_escaping.str(); static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without {}*, re2::StringPiece input(escaped_with_globs); re2::StringPiece matched; - std::ostringstream oss_for_replacing; + std::ostringstream oss_for_replacing; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss_for_replacing.exceptions(std::ios::failbit); size_t current_index = 0; while (RE2::FindAndConsume(&input, enum_or_range, &matched)) @@ -45,9 +48,8 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob size_t range_begin = 0; size_t range_end = 0; char point; - std::istringstream iss_range(buffer); - iss_range.exceptions(std::ios::failbit); - iss_range >> range_begin >> point >> point >> range_end; + ReadBufferFromString buf_range(buffer); + buf_range >> range_begin >> point >> point >> range_end; bool leading_zeros = buffer[0] == '0'; size_t num_len = std::to_string(range_end).size(); if (leading_zeros) @@ -71,20 +73,19 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob } oss_for_replacing << escaped_with_globs.substr(current_index); std::string almost_res = oss_for_replacing.str(); - std::ostringstream oss_final_processing; - oss_final_processing.exceptions(std::ios::failbit); + WriteBufferFromOwnString buf_final_processing; for (const auto & letter : almost_res) { if ((letter == '?') || (letter == '*')) { - oss_final_processing << "[^/]"; /// '?' is any symbol except '/' + buf_final_processing << "[^/]"; /// '?' is any symbol except '/' if (letter == '?') continue; } if ((letter == '.') || (letter == '{') || (letter == '}')) - oss_final_processing << '\\'; - oss_final_processing << letter; + buf_final_processing << '\\'; + buf_final_processing << letter; } - return oss_final_processing.str(); + return buf_final_processing.str(); } } diff --git a/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp b/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp index 4756043acbf..30aba9418ba 100644 --- a/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp +++ b/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp @@ -9,7 +9,8 @@ using namespace DB; TEST(Common, getMultipleValuesFromConfig) { - std::istringstream xml_isteam(R"END( + std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM + xml_isteam(R"END( 0 diff --git a/src/Common/tests/gtest_sensitive_data_masker.cpp b/src/Common/tests/gtest_sensitive_data_masker.cpp index 67ad5be2f52..b8125c77b9b 100644 --- a/src/Common/tests/gtest_sensitive_data_masker.cpp +++ b/src/Common/tests/gtest_sensitive_data_masker.cpp @@ -102,7 +102,8 @@ TEST(Common, SensitiveDataMasker) EXPECT_EQ(maskerbad.wipeSensitiveData(x), 0); { - std::istringstream xml_isteam(R"END( + std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM + xml_isteam(R"END( @@ -152,7 +153,8 @@ TEST(Common, SensitiveDataMasker) try { - std::istringstream xml_isteam_bad(R"END( + std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM + xml_isteam_bad(R"END( @@ -181,7 +183,8 @@ TEST(Common, SensitiveDataMasker) try { - std::istringstream xml_isteam_bad(R"END( + std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM + xml_isteam_bad(R"END( test @@ -203,7 +206,8 @@ TEST(Common, SensitiveDataMasker) try { - std::istringstream xml_isteam_bad(R"END( + std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM + xml_isteam_bad(R"END( test())( diff --git a/src/DataTypes/tests/gtest_data_type_get_common_type.cpp b/src/DataTypes/tests/gtest_data_type_get_common_type.cpp index 8212555e8bc..5c258f0b42e 100644 --- a/src/DataTypes/tests/gtest_data_type_get_common_type.cpp +++ b/src/DataTypes/tests/gtest_data_type_get_common_type.cpp @@ -26,7 +26,7 @@ static auto typeFromString(const std::string & str) static auto typesFromString(const std::string & str) { - std::istringstream data_types_stream(str); + std::istringstream data_types_stream(str); // STYLE_CHECK_ALLOW_STD_STRING_STREAM DataTypes data_types; std::string data_type; while (data_types_stream >> data_type) diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index 4c7cc5b4118..44924b69363 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -230,8 +231,7 @@ std::string DictionaryStructure::getKeyDescription() const if (id) return "UInt64"; - std::ostringstream out; - out.exceptions(std::ios::failbit); + WriteBufferFromOwnString out; out << '('; diff --git a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp index 62422124bd8..be0b0d7c6d2 100644 --- a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp +++ b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp @@ -18,7 +18,7 @@ static bool registered = false; static std::string configurationToString(const DictionaryConfigurationPtr & config) { const Poco::Util::XMLConfiguration * xml_config = dynamic_cast(config.get()); - std::ostringstream oss; + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); xml_config->save(oss); return oss.str(); diff --git a/src/IO/WriteHelpers.cpp b/src/IO/WriteHelpers.cpp index a0a2a45c791..61bfc281050 100644 --- a/src/IO/WriteHelpers.cpp +++ b/src/IO/WriteHelpers.cpp @@ -89,4 +89,12 @@ void writeProbablyBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf) writeProbablyQuotedStringImpl(s, buf, [](const StringRef & s_, WriteBuffer & buf_) { return writeBackQuotedStringMySQL(s_, buf_); }); } +void writePointerHex(const void * ptr, WriteBuffer & buf) +{ + writeString("0x", buf); + char hex_str[2 * sizeof(ptr)]; + writeHexUIntLowercase(reinterpret_cast(ptr), hex_str); + buf.write(hex_str, 2 * sizeof(ptr)); +} + } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index da68c4aded5..4a117fefbee 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1110,4 +1110,6 @@ struct PcgSerializer } }; +void writePointerHex(const void * ptr, WriteBuffer & buf); + } diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 4b7d3c2ea40..2a74c484187 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -148,4 +148,14 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const } } +void IAST::dumpTree(WriteBuffer & ostr, size_t indent) const +{ + String indent_str(indent, '-'); + ostr << indent_str << getID() << ", "; + writePointerHex(this, ostr); + writeChar('\n', ostr); + for (const auto & child : children) + child->dumpTree(ostr, indent + 1); +} + } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index d9fd71378f7..9428f312106 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -78,13 +78,7 @@ public: void updateTreeHash(SipHash & hash_state) const; virtual void updateTreeHashImpl(SipHash & hash_state) const; - void dumpTree(std::ostream & ostr, size_t indent = 0) const - { - String indent_str(indent, '-'); - ostr << indent_str << getID() << ", " << this << std::endl; - for (const auto & child : children) - child->dumpTree(ostr, indent + 1); - } + void dumpTree(WriteBuffer & ostr, size_t indent = 0) const; /** Check the depth of the tree. * If max_depth is specified and the depth is greater - throw an exception. diff --git a/src/Parsers/MySQL/tests/gtest_create_parser.cpp b/src/Parsers/MySQL/tests/gtest_create_parser.cpp index 1aaba8d67e4..554b3f0a67d 100644 --- a/src/Parsers/MySQL/tests/gtest_create_parser.cpp +++ b/src/Parsers/MySQL/tests/gtest_create_parser.cpp @@ -6,6 +6,7 @@ #include #include #include +#include using namespace DB; using namespace DB::MySQLParser; @@ -37,6 +38,7 @@ TEST(CreateTableParser, SS) ParserCreateQuery p_create_query; String input = "CREATE TABLE `test_table_1` (`a` int DEFAULT NULL, `b` int DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci"; ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); - ast->dumpTree(std::cerr); + WriteBufferFromOStream buf(std::cerr, 4096); + ast->dumpTree(buf); } diff --git a/src/Parsers/iostream_debug_helpers.cpp b/src/Parsers/iostream_debug_helpers.cpp index 66666f0dbfc..b74d337b22d 100644 --- a/src/Parsers/iostream_debug_helpers.cpp +++ b/src/Parsers/iostream_debug_helpers.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace DB { @@ -23,9 +25,10 @@ std::ostream & operator<<(std::ostream & stream, const Expected & what) std::ostream & operator<<(std::ostream & stream, const IAST & what) { - stream << "IAST{"; - what.dumpTree(stream); - stream << "}"; + WriteBufferFromOStream buf(stream, 4096); + buf << "IAST{"; + what.dumpTree(buf); + buf << "}"; return stream; } diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index 3fd173d8e18..0bef6699266 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -52,7 +52,6 @@ SRCS( ASTShowAccessEntitiesQuery.cpp ASTShowCreateAccessEntityQuery.cpp ASTShowGrantsQuery.cpp - ASTShowPrivilegesQuery.cpp ASTShowTablesQuery.cpp ASTSubquery.cpp ASTSystemQuery.cpp diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 8a416ade740..787059ddcc8 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -160,7 +160,7 @@ static void insertNumber(IColumn & column, WhichDataType type, T value) static std::string nodeToJson(avro::NodePtr root_node) { - std::ostringstream ss; + std::ostringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM ss.exceptions(std::ios::failbit); root_node->printJson(ss, 0); return ss.str(); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 4caeafc093f..78e43bad73a 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -17,6 +17,8 @@ #include #include #include +#include +#include #include @@ -33,8 +35,7 @@ namespace ErrorCodes String Range::toString() const { - std::stringstream str; - str.exceptions(std::ios::failbit); + WriteBufferFromOwnString str; if (!left_bounded) str << "(-inf, "; @@ -1559,19 +1560,18 @@ bool KeyCondition::mayBeTrueAfter( String KeyCondition::RPNElement::toString() const { - auto print_wrapped_column = [this](std::ostringstream & ss) + auto print_wrapped_column = [this](WriteBuffer & buf) { for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it) - ss << (*it)->getName() << "("; + buf << (*it)->getName() << "("; - ss << "column " << key_column; + buf << "column " << key_column; for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it) - ss << ")"; + buf << ")"; }; - std::ostringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString buf; switch (function) { case FUNCTION_AND: @@ -1585,24 +1585,24 @@ String KeyCondition::RPNElement::toString() const case FUNCTION_NOT_IN_SET: case FUNCTION_IN_SET: { - ss << "("; - print_wrapped_column(ss); - ss << (function == FUNCTION_IN_SET ? " in " : " notIn "); + buf << "("; + print_wrapped_column(buf); + buf << (function == FUNCTION_IN_SET ? " in " : " notIn "); if (!set_index) - ss << "unknown size set"; + buf << "unknown size set"; else - ss << set_index->size() << "-element set"; - ss << ")"; - return ss.str(); + buf << set_index->size() << "-element set"; + buf << ")"; + return buf.str(); } case FUNCTION_IN_RANGE: case FUNCTION_NOT_IN_RANGE: { - ss << "("; - print_wrapped_column(ss); - ss << (function == FUNCTION_NOT_IN_RANGE ? " not" : "") << " in " << range.toString(); - ss << ")"; - return ss.str(); + buf << "("; + print_wrapped_column(buf); + buf << (function == FUNCTION_NOT_IN_RANGE ? " not" : "") << " in " << range.toString(); + buf << ")"; + return buf.str(); } case ALWAYS_FALSE: return "false"; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7884dc7beaa..062a0611501 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3205,13 +3205,12 @@ void MergeTreeData::Transaction::rollbackPartsToTemporaryState() { if (!isEmpty()) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - ss << " Rollbacking parts state to temporary and removing from working set:"; + WriteBufferFromOwnString buf; + buf << " Rollbacking parts state to temporary and removing from working set:"; for (const auto & part : precommitted_parts) - ss << " " << part->relative_path; - ss << "."; - LOG_DEBUG(data.log, "Undoing transaction.{}", ss.str()); + buf << " " << part->relative_path; + buf << "."; + LOG_DEBUG(data.log, "Undoing transaction.{}", buf.str()); data.removePartsFromWorkingSetImmediatelyAndSetTemporaryState( DataPartsVector(precommitted_parts.begin(), precommitted_parts.end())); @@ -3224,13 +3223,12 @@ void MergeTreeData::Transaction::rollback() { if (!isEmpty()) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - ss << " Removing parts:"; + WriteBufferFromOwnString buf; + buf << " Removing parts:"; for (const auto & part : precommitted_parts) - ss << " " << part->relative_path; - ss << "."; - LOG_DEBUG(data.log, "Undoing transaction.{}", ss.str()); + buf << " " << part->relative_path; + buf << "."; + LOG_DEBUG(data.log, "Undoing transaction.{}", buf.str()); data.removePartsFromWorkingSet( DataPartsVector(precommitted_parts.begin(), precommitted_parts.end()), @@ -3760,15 +3758,15 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S if (out_reason && (settings.min_rows_for_wide_part != 0 || settings.min_bytes_for_wide_part != 0 || settings.min_rows_for_compact_part != 0 || settings.min_bytes_for_compact_part != 0)) { - std::ostringstream message; - message.exceptions(std::ios::failbit); - message << "Table can't create parts with adaptive granularity, but settings" - << " min_rows_for_wide_part = " << settings.min_rows_for_wide_part - << ", min_bytes_for_wide_part = " << settings.min_bytes_for_wide_part - << ", min_rows_for_compact_part = " << settings.min_rows_for_compact_part - << ", min_bytes_for_compact_part = " << settings.min_bytes_for_compact_part - << ". Parts with non-adaptive granularity can be stored only in Wide (default) format."; - *out_reason = message.str(); + *out_reason = fmt::format( + "Table can't create parts with adaptive granularity, but settings" + " min_rows_for_wide_part = {}" + ", min_bytes_for_wide_part = {}" + ", min_rows_for_compact_part = {}" + ", min_bytes_for_compact_part = {}" + ". Parts with non-adaptive granularity can be stored only in Wide (default) format.", + settings.min_rows_for_wide_part, settings.min_bytes_for_wide_part, + settings.min_rows_for_compact_part, settings.min_bytes_for_compact_part); } return false; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 9316a5742c1..0a266433ebc 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -49,6 +49,8 @@ #include #include +#include +#include #include @@ -175,19 +177,18 @@ UInt64 getMaximumFileNumber(const std::string & dir_path) std::string makeFormattedListOfShards(const ClusterPtr & cluster) { - std::ostringstream os; - os.exceptions(std::ios::failbit); + WriteBufferFromOwnString buf; bool head = true; - os << "["; + buf << "["; for (const auto & shard_info : cluster->getShardsInfo()) { - (head ? os : os << ", ") << shard_info.shard_num; + (head ? buf : buf << ", ") << shard_info.shard_num; head = false; } - os << "]"; + buf << "]"; - return os.str(); + return buf.str(); } ExpressionActionsPtr buildShardingKeyExpression(const ASTPtr & sharding_key, const Context & context, const NamesAndTypesList & columns, bool project) @@ -744,8 +745,7 @@ ClusterPtr StorageDistributed::getOptimizedCluster(const Context & context, cons UInt64 force = settings.force_optimize_skip_unused_shards; if (force) { - std::stringstream exception_message; - exception_message.exceptions(std::ios::failbit); + WriteBufferFromOwnString exception_message; if (!has_sharding_key) exception_message << "No sharding key"; else if (!sharding_key_is_usable) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ce9ebbd53b3..66c688f3195 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -254,18 +254,14 @@ Strings listFilesWithRegexpMatching(Aws::S3::S3Client & client, const S3::URI & outcome = client.ListObjectsV2(request); if (!outcome.IsSuccess()) { - std::ostringstream message; - message.exceptions(std::ios::failbit); - message << "Could not list objects in bucket " << quoteString(request.GetBucket()) - << " with prefix " << quoteString(request.GetPrefix()); - if (page > 1) - message << ", page " << std::to_string(page); + throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, page {}, S3 exception: {}, message: {}", + quoteString(request.GetBucket()), quoteString(request.GetPrefix()), page, + backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); - message << ", S3 exception: " + backQuote(outcome.GetError().GetExceptionName()) - << ", message: " + quoteString(outcome.GetError().GetMessage()); - - throw Exception(message.str(), ErrorCodes::S3_ERROR); + throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", + quoteString(request.GetBucket()), quoteString(request.GetPrefix()), + backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); } for (const auto & row : outcome.GetResult().GetContents()) diff --git a/src/Storages/System/StorageSystemUsers.cpp b/src/Storages/System/StorageSystemUsers.cpp index 70c67683b25..675fee84746 100644 --- a/src/Storages/System/StorageSystemUsers.cpp +++ b/src/Storages/System/StorageSystemUsers.cpp @@ -96,7 +96,7 @@ void StorageSystemUsers::fillData(MutableColumns & res_columns, const Context & auth_params_json.set("server", authentication.getServerName()); - std::ostringstream oss; + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(auth_params_json, oss); const auto str = oss.str(); diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 821fbc4b279..0fb418e8413 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -133,16 +133,14 @@ std::string readData(DB::StoragePtr & table, const DB::Context & context) tryRegisterFormats(); - std::ostringstream ss; - ss.exceptions(std::ios::failbit); - WriteBufferFromOStream out_buf(ss); + WriteBufferFromOwnString out_buf; BlockOutputStreamPtr output = FormatFactory::instance().getOutput("Values", out_buf, sample, context); copyData(*in, *output); output->flush(); - return ss.str(); + return out_buf.str(); } TYPED_TEST(StorageLogTest, testReadWrite) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 69d948c87fe..65ed4cec67a 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -107,4 +107,4 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P ' $' | grep -P '.' && echo "^ Trailing whitespaces." # Forbid stringstream because it's easy to use them incorrectly and hard to debug possible issues -find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep 'std::ostringstream\|std::istringstream' && echo "Use WriteBufferFromString or ReadBufferFromString instead of std::ostringstream or std::istringstream" +find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep 'std::ostringstream\|std::istringstream' | grep -v "STYLE_CHECK_ALLOW_STD_STRING_STREAM" && echo "Use WriteBufferFromString or ReadBufferFromString instead of std::ostringstream or std::istringstream" From fdce810237a3cab731ebbd4a4b9af73cfd06f9d7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Nov 2020 22:32:18 +0300 Subject: [PATCH 50/94] 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 51/94] 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 52/94] 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 53/94] 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 54/94] 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 55/94] 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 f999ea2a6f123a962e2fa2e7edd38bb6b0f6e1c9 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Tue, 10 Nov 2020 01:52:22 +0300 Subject: [PATCH 56/94] renamed files, added new library, changed error codes, added tests for file() function --- .gitmodules | 4 ++ CHANGELOG.md | 1 - contrib/fast-lzma2 | 1 + src/CMakeLists.txt | 7 +++ src/Common/ErrorCodes.cpp | 6 +-- src/IO/CompressionMethod.cpp | 50 +++++++++---------- ...uffer.cpp => LZMADeflatingWriteBuffer.cpp} | 23 ++++++--- ...iteBuffer.h => LZMADeflatingWriteBuffer.h} | 9 ++-- ...Buffer.cpp => LZMAInflatingReadBuffer.cpp} | 18 +++++-- ...ReadBuffer.h => LZMAInflatingReadBuffer.h} | 11 ++-- src/IO/tests/lzma_buffers.cpp | 26 +++++----- .../01059_storage_file_brotli.reference | 4 +- .../0_stateless/01059_storage_file_brotli.sql | 11 +++- 13 files changed, 105 insertions(+), 66 deletions(-) create mode 160000 contrib/fast-lzma2 rename src/IO/{LzmaWriteBuffer.cpp => LZMADeflatingWriteBuffer.cpp} (85%) rename src/IO/{LzmaWriteBuffer.h => LZMADeflatingWriteBuffer.h} (73%) rename src/IO/{LzmaReadBuffer.cpp => LZMAInflatingReadBuffer.cpp} (81%) rename src/IO/{LzmaReadBuffer.h => LZMAInflatingReadBuffer.h} (68%) diff --git a/.gitmodules b/.gitmodules index ecaf806067c..e489444d46c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -17,6 +17,7 @@ [submodule "contrib/zlib-ng"] path = contrib/zlib-ng url = https://github.com/ClickHouse-Extras/zlib-ng.git + branch = clickhouse [submodule "contrib/googletest"] path = contrib/googletest url = https://github.com/google/googletest.git @@ -193,3 +194,6 @@ [submodule "contrib/xz"] path = contrib/xz url = https://github.com/xz-mirror/xz +[submodule "contrib/fast-lzma2"] + path = contrib/fast-lzma2 + url = https://github.com/conor42/fast-lzma2 diff --git a/CHANGELOG.md b/CHANGELOG.md index 1242bd2e7b4..457346aff9a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -219,7 +219,6 @@ * Added column transformers `EXCEPT`, `REPLACE`, `APPLY`, which can be applied to the list of selected columns (after `*` or `COLUMNS(...)`). For example, you can write `SELECT * EXCEPT(URL) REPLACE(number + 1 AS number)`. Another example: `select * apply(length) apply(max) from wide_string_table` to find out the maxium length of all string columns. [#14233](https://github.com/ClickHouse/ClickHouse/pull/14233) ([Amos Bird](https://github.com/amosbird)). * Added an aggregate function `rankCorr` which computes a rank correlation coefficient. [#11769](https://github.com/ClickHouse/ClickHouse/pull/11769) ([antikvist](https://github.com/antikvist)) [#14411](https://github.com/ClickHouse/ClickHouse/pull/14411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Added table function `view` which turns a subquery into a table object. This helps passing queries around. For instance, it can be used in remote/cluster table functions. [#12567](https://github.com/ClickHouse/ClickHouse/pull/12567) ([Amos Bird](https://github.com/amosbird)). -* Added support for `xz` compression format. This enables using `*.xz` files in `table()` function. [#16578](https://github.com/ClickHouse/ClickHouse/pull/16578) ([Abi Palagashvili](https://github.com/fibersel)) #### Bug Fix diff --git a/contrib/fast-lzma2 b/contrib/fast-lzma2 new file mode 160000 index 00000000000..ded964d203c --- /dev/null +++ b/contrib/fast-lzma2 @@ -0,0 +1 @@ +Subproject commit ded964d203cabe1a572d2c813c55e8a94b4eda48 diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 7a8fe586e11..6839b06677d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -336,6 +336,13 @@ if (LZMA_LIBRARY) target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${LZMA_INCLUDE_DIR}) endif() +set (FAST_LZMA_LIBRARY fast-lzma2) +set (FAST_LZMA_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/fast-lzma2/) +if (FAST_LZMA_LIBRARY) + target_link_libraries (clickhouse_common_io PUBLIC ${FAST_LZMA_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${FAST_LZMA_INCLUDE_DIR}) +endif() + if (USE_ICU) dbms_target_link_libraries (PRIVATE ${ICU_LIBRARIES}) dbms_target_include_directories (SYSTEM PRIVATE ${ICU_INCLUDE_DIRS}) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b12623c73ff..82084976a82 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -519,13 +519,13 @@ M(550, CONDITIONAL_TREE_PARENT_NOT_FOUND) \ M(551, ILLEGAL_PROJECTION_MANIPULATOR) \ M(552, UNRECOGNIZED_ARGUMENTS) \ + M(553, LZMA_STREAM_ENCODER_FAILED) \ + M(554, LZMA_STREAM_DECODER_FAILED) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ - M(1002, UNKNOWN_EXCEPTION) \ - M(1003, LZMA_STREAM_ENCODER_FAILED) \ - M(1004, LZMA_STREAM_DECODER_FAILED) + M(1002, UNKNOWN_EXCEPTION) /* See END */ diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index 0bf390d92ca..ec278b5d71f 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -1,13 +1,13 @@ #include -#include -#include -#include -#include #include #include -#include -#include +#include +#include +#include +#include +#include +#include #if !defined(ARCADIA_BUILD) # include @@ -16,7 +16,6 @@ namespace DB { - namespace ErrorCodes { extern const int NOT_IMPLEMENTED; @@ -27,11 +26,16 @@ std::string toContentEncodingName(CompressionMethod method) { switch (method) { - case CompressionMethod::Gzip: return "gzip"; - case CompressionMethod::Zlib: return "deflate"; - case CompressionMethod::Brotli: return "br"; - case CompressionMethod::Xz: return "xz"; - case CompressionMethod::None: return ""; + case CompressionMethod::Gzip: + return "gzip"; + case CompressionMethod::Zlib: + return "deflate"; + case CompressionMethod::Brotli: + return "br"; + case CompressionMethod::Xz: + return "xz"; + case CompressionMethod::None: + return ""; } __builtin_unreachable(); } @@ -55,20 +59,19 @@ CompressionMethod chooseCompressionMethod(const std::string & path, const std::s return CompressionMethod::Zlib; if (*method_str == "brotli" || *method_str == "br") return CompressionMethod::Brotli; + if (*method_str == "LZMA" || *method_str == "xz") + return CompressionMethod::Xz; if (hint.empty() || hint == "auto" || hint == "none") return CompressionMethod::None; - throw Exception("Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'br' are supported as compression methods", + throw Exception( + "Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'br', 'xz' are supported as compression methods", ErrorCodes::NOT_IMPLEMENTED); } std::unique_ptr wrapReadBufferWithCompressionMethod( - std::unique_ptr nested, - CompressionMethod method, - size_t buf_size, - char * existing_memory, - size_t alignment) + std::unique_ptr nested, CompressionMethod method, size_t buf_size, char * existing_memory, size_t alignment) { if (method == CompressionMethod::Gzip || method == CompressionMethod::Zlib) return std::make_unique(std::move(nested), method, buf_size, existing_memory, alignment); @@ -77,7 +80,7 @@ std::unique_ptr wrapReadBufferWithCompressionMethod( return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); #endif if (method == CompressionMethod::Xz) - return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); + return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); if (method == CompressionMethod::None) return nested; @@ -87,12 +90,7 @@ std::unique_ptr wrapReadBufferWithCompressionMethod( std::unique_ptr wrapWriteBufferWithCompressionMethod( - std::unique_ptr nested, - CompressionMethod method, - int level, - size_t buf_size, - char * existing_memory, - size_t alignment) + std::unique_ptr nested, CompressionMethod method, int level, size_t buf_size, char * existing_memory, size_t alignment) { if (method == DB::CompressionMethod::Gzip || method == CompressionMethod::Zlib) return std::make_unique(std::move(nested), method, level, buf_size, existing_memory, alignment); @@ -101,6 +99,8 @@ std::unique_ptr wrapWriteBufferWithCompressionMethod( if (method == DB::CompressionMethod::Brotli) return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); #endif + if (method == CompressionMethod::Xz) + return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); if (method == CompressionMethod::None) return nested; diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp similarity index 85% rename from src/IO/LzmaWriteBuffer.cpp rename to src/IO/LZMADeflatingWriteBuffer.cpp index 8439fc624d4..66cd11c13d6 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB @@ -8,11 +8,15 @@ namespace ErrorCodes extern const int LZMA_STREAM_ENCODER_FAILED; } -LzmaWriteBuffer::LzmaWriteBuffer( +LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( std::unique_ptr out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), out(std::move(out_)) { - lstr = LZMA_STREAM_INIT; + // FL2_createCStreamMt(number of threads, flag of two dictionaries usage) + lstr = FL2_createCStreamMt(2, 0); + /* size_t res = */ FL2_initCStream(lstr, compression_level); + + /*lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; lstr.next_in = nullptr; lstr.avail_in = 0; @@ -37,15 +41,16 @@ LzmaWriteBuffer::LzmaWriteBuffer( throw Exception( std::string("lzma stream encoder init failed: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + */ } -LzmaWriteBuffer::~LzmaWriteBuffer() +LZMADeflatingWriteBuffer::~LZMADeflatingWriteBuffer() { try { finish(); - lzma_end(&lstr); + //lzma_end(&lstr); } catch (...) { @@ -53,8 +58,9 @@ LzmaWriteBuffer::~LzmaWriteBuffer() } } -void LzmaWriteBuffer::nextImpl() +void LZMADeflatingWriteBuffer::nextImpl() { + /* if (!offset()) return; @@ -82,11 +88,13 @@ void LzmaWriteBuffer::nextImpl() ErrorCodes::LZMA_STREAM_ENCODER_FAILED); } while (lstr.avail_in > 0 || lstr.avail_out == 0); + */ } -void LzmaWriteBuffer::finish() +void LZMADeflatingWriteBuffer::finish() { + /* if (finished) return; @@ -114,5 +122,6 @@ void LzmaWriteBuffer::finish() ErrorCodes::LZMA_STREAM_ENCODER_FAILED); } while (lstr.avail_out == 0); + */ } } diff --git a/src/IO/LzmaWriteBuffer.h b/src/IO/LZMADeflatingWriteBuffer.h similarity index 73% rename from src/IO/LzmaWriteBuffer.h rename to src/IO/LZMADeflatingWriteBuffer.h index d59595dab23..aadf15ec6dd 100644 --- a/src/IO/LzmaWriteBuffer.h +++ b/src/IO/LZMADeflatingWriteBuffer.h @@ -4,14 +4,15 @@ #include #include +#include namespace DB { /// Performs compression using lzma library and writes compressed data to out_ WriteBuffer. -class LzmaWriteBuffer : public BufferWithOwnMemory +class LZMADeflatingWriteBuffer : public BufferWithOwnMemory { public: - LzmaWriteBuffer( + LZMADeflatingWriteBuffer( std::unique_ptr out_, int compression_level, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, @@ -20,13 +21,13 @@ public: void finish(); - ~LzmaWriteBuffer() override; + ~LZMADeflatingWriteBuffer() override; private: void nextImpl() override; std::unique_ptr out; - lzma_stream lstr; + FL2_CStream * lstr; bool finished = false; }; } diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LZMAInflatingReadBuffer.cpp similarity index 81% rename from src/IO/LzmaReadBuffer.cpp rename to src/IO/LZMAInflatingReadBuffer.cpp index 22fda48b3c6..09aa31b7f43 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LZMAInflatingReadBuffer.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { @@ -6,9 +6,13 @@ namespace ErrorCodes { extern const int LZMA_STREAM_DECODER_FAILED; } -LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) +LZMAInflatingReadBuffer::LZMAInflatingReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)), eof(false) { + // FL2_createDStreamMt(number of threads) + lstr = FL2_createDStreamMt(2); + /* size_t res = */ FL2_initDStream(lstr); + /* lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; lstr.next_in = nullptr; @@ -26,15 +30,17 @@ LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_DECODER_FAILED); + */ } -LzmaReadBuffer::~LzmaReadBuffer() +LZMAInflatingReadBuffer::~LZMAInflatingReadBuffer() { - lzma_end(&lstr); + //lzma_end(&lstr); } -bool LzmaReadBuffer::nextImpl() +bool LZMAInflatingReadBuffer::nextImpl() { + /* if (eof) return false; @@ -77,5 +83,7 @@ bool LzmaReadBuffer::nextImpl() LZMA_VERSION_STRING); return true; + */ + return true; } } diff --git a/src/IO/LzmaReadBuffer.h b/src/IO/LZMAInflatingReadBuffer.h similarity index 68% rename from src/IO/LzmaReadBuffer.h rename to src/IO/LZMAInflatingReadBuffer.h index 5f936475ee1..4a9893e5b4c 100644 --- a/src/IO/LzmaReadBuffer.h +++ b/src/IO/LZMAInflatingReadBuffer.h @@ -1,10 +1,10 @@ #pragma once #include -#include #include #include +#include namespace DB { @@ -12,22 +12,23 @@ namespace ErrorCodes { } -class LzmaReadBuffer : public BufferWithOwnMemory +class LZMAInflatingReadBuffer : public BufferWithOwnMemory { public: - LzmaReadBuffer( + LZMAInflatingReadBuffer( std::unique_ptr in_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0); - ~LzmaReadBuffer() override; + ~LZMAInflatingReadBuffer() override; private: bool nextImpl() override; std::unique_ptr in; - lzma_stream lstr; + FL2_DStream * lstr; + bool eof; }; } diff --git a/src/IO/tests/lzma_buffers.cpp b/src/IO/tests/lzma_buffers.cpp index 1f691fa09f7..7eb6bf8b81c 100644 --- a/src/IO/tests/lzma_buffers.cpp +++ b/src/IO/tests/lzma_buffers.cpp @@ -1,13 +1,13 @@ -#include #include +#include -#include -#include +#include +#include #include -#include -#include -#include #include +#include +#include +#include int main(int, char **) try @@ -18,8 +18,9 @@ try Stopwatch stopwatch; { - auto buf = std::make_unique("test_lzma_buffers.xz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); - DB::LzmaWriteBuffer lzma_buf(std::move(buf), /*compression level*/ 3); + auto buf + = std::make_unique("test_lzma_buffers.xz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); + DB::LZMADeflatingWriteBuffer lzma_buf(std::move(buf), /*compression level*/ 3); stopwatch.restart(); for (size_t i = 0; i < n; ++i) @@ -32,13 +33,12 @@ try stopwatch.stop(); std::cout << "Writing done. Elapsed: " << stopwatch.elapsedSeconds() << " s." - << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" - << std::endl; + << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" << std::endl; } { auto buf = std::make_unique("test_lzma_buffers.xz"); - DB::LzmaReadBuffer lzma_buf(std::move(buf)); + DB::LZMAInflatingReadBuffer lzma_buf(std::move(buf)); stopwatch.restart(); for (size_t i = 0; i < n; ++i) @@ -52,9 +52,7 @@ try } stopwatch.stop(); std::cout << "Reading done. Elapsed: " << stopwatch.elapsedSeconds() << " s." - << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" - << std::endl; - + << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" << std::endl; } return 0; diff --git a/tests/queries/0_stateless/01059_storage_file_brotli.reference b/tests/queries/0_stateless/01059_storage_file_brotli.reference index 6c545e9faec..aae55b2873c 100644 --- a/tests/queries/0_stateless/01059_storage_file_brotli.reference +++ b/tests/queries/0_stateless/01059_storage_file_brotli.reference @@ -1,5 +1,7 @@ 1000000 999999 1000000 999999 -2000000 999999 +1000000 999999 +3000000 999999 1 255 1 255 +1 255 \ No newline at end of file diff --git a/tests/queries/0_stateless/01059_storage_file_brotli.sql b/tests/queries/0_stateless/01059_storage_file_brotli.sql index e7d5a87b2af..eba61e4450f 100644 --- a/tests/queries/0_stateless/01059_storage_file_brotli.sql +++ b/tests/queries/0_stateless/01059_storage_file_brotli.sql @@ -15,8 +15,17 @@ SELECT count(), max(x) FROM file; DROP TABLE file; -SELECT count(), max(x) FROM file('data{1,2}.tsv.{gz,br}', TSV, 'x UInt64'); +CREATE TABLE file (x UInt64) ENGINE = File(TSV, 'data3.tsv.xz'); +TRUNCATE TABLE file; + +INSERT INTO file SELECT * FROM numbers(1000000); +SELECT count(), max(x) FROM file; + +DROP TABLE file; + +SELECT count(), max(x) FROM file('data{1,2,3}.tsv.{gz,br,xz}', TSV, 'x UInt64'); -- check that they are compressed SELECT count() < 1000000, max(x) FROM file('data1.tsv.br', RowBinary, 'x UInt8', 'none'); SELECT count() < 3000000, max(x) FROM file('data2.tsv.gz', RowBinary, 'x UInt8', 'none'); +SELECT count() < 1000000, max(x) FROM file('data3.tsv.xz', RowBinary, 'x UInt8', 'none'); From ca51cf9235316ffee17af7cfec5b7e2a611bc3d5 Mon Sep 17 00:00:00 2001 From: fenglv Date: Tue, 10 Nov 2020 07:20:50 +0000 Subject: [PATCH 57/94] 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 435f410aaa393667f6d107e0054db705d679c740 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Tue, 10 Nov 2020 10:35:27 +0300 Subject: [PATCH 58/94] Restart C From aa8e6db786fd7007e71e0071cde9fe9018987045 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 3 Nov 2020 15:52:31 +0800 Subject: [PATCH 59/94] reload auxiliary zookeepers configuration --- programs/server/Server.cpp | 2 + src/Interpreters/Context.cpp | 42 +++++++-- src/Interpreters/Context.h | 3 + .../__init__.py | 0 .../configs/config.xml | 31 +++++++ .../configs/users.xml | 23 +++++ .../configs/zookeeper.xml | 17 ++++ .../test_reload_auxiliary_zookeepers/test.py | 89 +++++++++++++++++++ 8 files changed, 200 insertions(+), 7 deletions(-) create mode 100644 tests/integration/test_reload_auxiliary_zookeepers/__init__.py create mode 100644 tests/integration/test_reload_auxiliary_zookeepers/configs/config.xml create mode 100644 tests/integration/test_reload_auxiliary_zookeepers/configs/users.xml create mode 100644 tests/integration/test_reload_auxiliary_zookeepers/configs/zookeeper.xml create mode 100644 tests/integration/test_reload_auxiliary_zookeepers/test.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ed18793a537..a481bc020d8 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -568,6 +568,8 @@ int Server::main(const std::vector & /*args*/) if (config->has("zookeeper")) global_context->reloadZooKeeperIfChanged(config); + global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config); + global_context->updateStorageConfiguration(*config); }, /* already_loaded = */ true); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a22eacd9a49..73604f93d12 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -302,9 +302,11 @@ struct ContextShared mutable std::mutex zookeeper_mutex; mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper. + ConfigurationPtr zookeeper_config; /// Stores zookeeper configs mutable std::mutex auxiliary_zookeepers_mutex; mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. + ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs String interserver_io_host; /// The host name by which this server is available for other servers. UInt16 interserver_io_port = 0; /// and port. @@ -364,8 +366,7 @@ struct ContextShared /// Initialized on demand (on distributed storages initialization) since Settings should be initialized std::unique_ptr clusters; ConfigurationPtr clusters_config; /// Stores updated configs - ConfigurationPtr zookeeper_config; /// Stores zookeeper configs - mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config + mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config #if USE_EMBEDDED_COMPILER std::shared_ptr compiled_expression_cache; @@ -1498,10 +1499,16 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const auto zookeeper = shared->auxiliary_zookeepers.find(name); if (zookeeper == shared->auxiliary_zookeepers.end()) { - if (!getConfigRef().has("auxiliary_zookeepers." + name)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown auxiliary ZooKeeper name '{}'. If it's required it can be added to the section in config.xml", name); + const auto & config = shared->auxiliary_zookeepers_config ? *shared->auxiliary_zookeepers_config : getConfigRef(); + if (!config.has("auxiliary_zookeepers." + name)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Unknown auxiliary ZooKeeper name '{}'. If it's required it can be added to the section in " + "config.xml", + name); - zookeeper->second = std::make_shared(getConfigRef(), "auxiliary_zookeepers." + name); + zookeeper + = shared->auxiliary_zookeepers.emplace(name, std::make_shared(config, "auxiliary_zookeepers." + name)).first; } else if (zookeeper->second->expired()) zookeeper->second = zookeeper->second->startNewSession(); @@ -1515,17 +1522,38 @@ void Context::resetZooKeeper() const shared->zookeeper.reset(); } +static void reloadZooKeeperIfChangedImpl(const ConfigurationPtr & config, const std::string & config_name, zkutil::ZooKeeperPtr & zk) +{ + if (!zk || zk->configChanged(*config, config_name)) + zk = std::make_shared(*config, config_name); +} + void Context::reloadZooKeeperIfChanged(const ConfigurationPtr & config) const { std::lock_guard lock(shared->zookeeper_mutex); shared->zookeeper_config = config; + reloadZooKeeperIfChangedImpl(config, "zookeeper", shared->zookeeper); +} - if (!shared->zookeeper || shared->zookeeper->configChanged(*config, "zookeeper")) +void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config) +{ + std::lock_guard lock(shared->auxiliary_zookeepers_mutex); + + shared->auxiliary_zookeepers_config = config; + + for (auto it = shared->auxiliary_zookeepers.begin(); it != shared->auxiliary_zookeepers.end();) { - shared->zookeeper = std::make_shared(*config, "zookeeper"); + if (!config->has("auxiliary_zookeepers." + it->first)) + it = shared->auxiliary_zookeepers.erase(it); + else + { + reloadZooKeeperIfChangedImpl(config, "auxiliary_zookeepers." + it->first, it->second); + ++it; + } } } + bool Context::hasZooKeeper() const { return getConfigRef().has("zookeeper"); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ec42880309f..a2b61b29514 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -487,6 +487,9 @@ public: std::shared_ptr getZooKeeper() const; /// Same as above but return a zookeeper connection from auxiliary_zookeepers configuration entry. std::shared_ptr getAuxiliaryZooKeeper(const String & name) const; + + /// Set auxiliary zookeepers configuration at server starting or configuration reloading. + void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config); /// Has ready or expired ZooKeeper bool hasZooKeeper() const; /// Reset current zookeeper session. Do not create a new one. diff --git a/tests/integration/test_reload_auxiliary_zookeepers/__init__.py b/tests/integration/test_reload_auxiliary_zookeepers/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_reload_auxiliary_zookeepers/configs/config.xml b/tests/integration/test_reload_auxiliary_zookeepers/configs/config.xml new file mode 100644 index 00000000000..b5e5495c096 --- /dev/null +++ b/tests/integration/test_reload_auxiliary_zookeepers/configs/config.xml @@ -0,0 +1,31 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + 9000 + 127.0.0.1 + + + + true + none + + AcceptCertificateHandler + + + + + 500 + 5368709120 + ./clickhouse/ + users.xml + + 1 + 1 + diff --git a/tests/integration/test_reload_auxiliary_zookeepers/configs/users.xml b/tests/integration/test_reload_auxiliary_zookeepers/configs/users.xml new file mode 100644 index 00000000000..6061af8e33d --- /dev/null +++ b/tests/integration/test_reload_auxiliary_zookeepers/configs/users.xml @@ -0,0 +1,23 @@ + + + + + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_reload_auxiliary_zookeepers/configs/zookeeper.xml b/tests/integration/test_reload_auxiliary_zookeepers/configs/zookeeper.xml new file mode 100644 index 00000000000..8157cc1d30b --- /dev/null +++ b/tests/integration/test_reload_auxiliary_zookeepers/configs/zookeeper.xml @@ -0,0 +1,17 @@ + + + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 2000 + + diff --git a/tests/integration/test_reload_auxiliary_zookeepers/test.py b/tests/integration/test_reload_auxiliary_zookeepers/test.py new file mode 100644 index 00000000000..92c66c890fc --- /dev/null +++ b/tests/integration/test_reload_auxiliary_zookeepers/test.py @@ -0,0 +1,89 @@ +import time +import pytest +import os + +from helpers.cluster import ClickHouseCluster +from helpers.client import QueryRuntimeException +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__, zookeeper_config_path="configs/zookeeper.xml") +node = cluster.add_instance("node", with_zookeeper=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_reload_auxiliary_zookeepers(start_cluster): + + node.query( + "CREATE TABLE simple (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', 'node') ORDER BY tuple() PARTITION BY date;" + ) + node.query("INSERT INTO simple VALUES ('2020-08-27', 1)") + + node.query( + "CREATE TABLE simple2 (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/1/simple', 'node') ORDER BY tuple() PARTITION BY date;" + ) + + # Add an auxiliary zookeeper + new_config = """ + + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 2000 + + + + + zoo1 + 2181 + + + zoo2 + 2181 + + + +""" + node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) + + # Hopefully it has finished the configuration reload + time.sleep(2) + + node.query( + "ALTER TABLE simple2 FETCH PARTITION '2020-08-27' FROM 'zookeeper2:/clickhouse/tables/0/simple';" + ) + node.query("ALTER TABLE simple2 ATTACH PARTITION '2020-08-27';") + assert node.query("SELECT id FROM simple2").strip() == "1" + + new_config = """ + + + zoo2 + 2181 + + 2000 + +""" + node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) + time.sleep(2) + with pytest.raises(QueryRuntimeException): + node.query( + "ALTER TABLE simple2 FETCH PARTITION '2020-08-27' FROM 'zookeeper2:/clickhouse/tables/0/simple';" + ) + assert node.query("SELECT id FROM simple2").strip() == "1" From 59c0a739c4a3ff4d157f64b5837f57c7aa448895 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 12:18:51 +0300 Subject: [PATCH 60/94] 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 61/94] 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 046830967c80542535f64912fdab75ad9a6bc503 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 13:26:26 +0300 Subject: [PATCH 62/94] 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 55631e442bd4e69bfc23fa2a9584e0bf6ee07211 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 10 Nov 2020 14:43:22 +0300 Subject: [PATCH 63/94] fix bug in WriteBufferFromVector --- src/IO/WriteBufferFromVector.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromVector.h b/src/IO/WriteBufferFromVector.h index 047e16643ce..54f43b6b591 100644 --- a/src/IO/WriteBufferFromVector.h +++ b/src/IO/WriteBufferFromVector.h @@ -36,8 +36,10 @@ private: throw Exception("WriteBufferFromVector is finished", ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER); size_t old_size = vector.size(); + /// pos may not be equal to vector.data() + old_size, because WriteBuffer::next() can be used to flush data + size_t pos_offset = pos - reinterpret_cast(vector.data()); vector.resize(old_size * size_multiplier); - internal_buffer = Buffer(reinterpret_cast(vector.data() + old_size), reinterpret_cast(vector.data() + vector.size())); + internal_buffer = Buffer(reinterpret_cast(vector.data() + pos_offset), reinterpret_cast(vector.data() + vector.size())); working_buffer = internal_buffer; } From 07fe3a6347f7d5a83d4a374adac9a531062c5b59 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 15:14:05 +0300 Subject: [PATCH 64/94] 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 65/94] 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 66/94] 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 67/94] 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 68/94] 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 69/94] 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 70/94] 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 71/94] 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 72/94] 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 From b94cc5c4e53c3d275d287e0b6b71e37d9421d339 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 10 Nov 2020 21:22:26 +0300 Subject: [PATCH 73/94] remove more stringstreams --- base/mysqlxx/Query.h | 2 +- programs/client/Client.cpp | 2 + programs/client/Suggest.cpp | 2 +- programs/client/TestHint.h | 2 +- programs/copier/ClusterCopier.cpp | 2 +- programs/copier/Internals.cpp | 2 +- programs/copier/TaskTableAndShard.h | 8 +- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 8 +- .../AggregateFunctionRetention.h | 2 - .../AggregateFunctionTimeSeriesGroupSum.h | 2 - .../AggregateFunctionWindowFunnel.h | 2 - src/AggregateFunctions/ReservoirSampler.h | 1 - .../ReservoirSamplerDeterministic.h | 1 - src/Columns/tests/gtest_weak_hash_32.cpp | 2 +- src/Common/Config/ConfigProcessor.cpp | 6 +- src/Common/Exception.cpp | 6 +- src/Common/MemoryTracker.cpp | 30 ++---- src/Common/ShellCommand.cpp | 4 +- src/Common/StackTrace.cpp | 8 +- src/Common/StudentTTest.cpp | 2 +- src/Common/ThreadStatus.cpp | 10 +- src/Common/UInt128.h | 1 - src/Common/XDBCBridgeHelper.h | 4 - src/Common/ZooKeeper/TestKeeper.cpp | 2 +- src/Common/checkStackSize.cpp | 12 +-- src/Common/formatReadable.cpp | 2 - src/Compression/CompressedReadBufferBase.cpp | 14 ++- src/Compression/tests/compressed_buffer.cpp | 5 +- src/Core/MySQL/IMySQLReadPacket.cpp | 8 +- src/Core/MySQL/IMySQLWritePacket.cpp | 12 ++- src/Core/MySQL/MySQLReplication.cpp | 101 +++++++++--------- src/Core/MySQL/MySQLReplication.h | 22 ++-- src/Core/SortDescription.h | 5 +- src/Core/tests/mysql_protocol.cpp | 21 ++-- .../CheckConstraintsBlockOutputStream.cpp | 38 +++---- src/DataStreams/IBlockInputStream.cpp | 10 +- src/DataStreams/IBlockInputStream.h | 2 +- .../MergingSortedBlockInputStream.cpp | 2 - src/DataStreams/MongoDBBlockInputStream.cpp | 1 - .../DataTypeCustomSimpleAggregateFunction.cpp | 3 +- src/DataTypes/DataTypesDecimal.cpp | 5 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 27 ++--- src/Functions/abtesting.cpp | 1 - src/IO/HTTPCommon.cpp | 2 +- src/IO/MySQLPacketPayloadReadBuffer.cpp | 1 - src/IO/S3/PocoHTTPClient.cpp | 5 +- src/IO/tests/gtest_bit_io.cpp | 2 +- src/IO/tests/hashing_read_buffer.cpp | 2 +- src/IO/tests/limit_read_buffer2.cpp | 2 +- src/IO/tests/write_buffer.cpp | 2 +- src/Interpreters/Context.cpp | 28 +++-- src/Interpreters/ExpressionActions.cpp | 13 +-- src/Interpreters/InterpreterExplainQuery.cpp | 2 - ...InterpreterShowCreateAccessEntityQuery.cpp | 1 - .../InterpreterShowCreateQuery.cpp | 2 - .../InterpreterShowTablesQuery.cpp | 25 ++--- src/Interpreters/QueryAliasesVisitor.cpp | 3 - src/Interpreters/Set.cpp | 8 +- src/Interpreters/TreeRewriter.cpp | 3 +- src/Interpreters/executeQuery.cpp | 8 +- src/Parsers/ASTCreateRowPolicyQuery.cpp | 1 - src/Parsers/ASTIdentifier.cpp | 1 - src/Parsers/ASTWithAlias.cpp | 1 - src/Parsers/DumpASTNode.h | 1 - src/Parsers/IAST.cpp | 1 - src/Parsers/IAST.h | 2 - src/Parsers/formatAST.cpp | 1 - src/Parsers/formatAST.h | 22 ++-- .../Formats/Impl/MySQLOutputFormat.cpp | 2 - .../Formats/Impl/PrettyBlockOutputFormat.cpp | 14 +-- .../Impl/PrettyCompactBlockOutputFormat.cpp | 16 ++- .../Algorithms/CollapsingSortedAlgorithm.cpp | 4 +- src/Server/ReplicasStatusHandler.cpp | 5 +- src/Server/TCPHandler.cpp | 10 +- .../DistributedBlockOutputStream.cpp | 3 +- src/Storages/Kafka/StorageKafka.cpp | 10 +- src/Storages/MergeTree/KeyCondition.h | 1 - .../MergeTree/MergeTreeDataPartChecksum.cpp | 9 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 10 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 3 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 9 +- src/Storages/StorageDictionary.cpp | 14 +-- src/Storages/StorageInMemoryMetadata.cpp | 4 +- src/Storages/StorageMergeTree.cpp | 28 ++--- src/Storages/StorageReplicatedMergeTree.cpp | 67 +++++------- src/Storages/StorageS3.cpp | 2 - .../transformQueryForExternalDatabase.cpp | 1 - src/TableFunctions/TableFunctionRemote.cpp | 10 +- utils/check-mysql-binlog/main.cpp | 21 ++-- utils/check-style/check-style | 2 +- .../ProtobufDelimitedMessagesSerializer.cpp | 4 +- utils/zookeeper-cli/zookeeper-cli.cpp | 2 +- 93 files changed, 323 insertions(+), 471 deletions(-) diff --git a/base/mysqlxx/Query.h b/base/mysqlxx/Query.h index d0a905e2031..1d3ab9678d5 100644 --- a/base/mysqlxx/Query.h +++ b/base/mysqlxx/Query.h @@ -77,7 +77,7 @@ public: private: Connection * conn; - std::ostringstream query_buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + std::ostringstream query_buf; void executeImpl(); }; diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 98c1463ab23..c7884b813cf 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1189,6 +1189,7 @@ private: fprintf(stderr, "dump after fuzz:\n"); WriteBufferFromOStream cerr_buf(std::cerr, 4096); fuzz_base->dumpTree(cerr_buf); + cerr_buf.next(); fmt::print(stderr, "IAST::clone() is broken for some AST node. This is a bug. The original AST ('dump before fuzz') and its cloned copy ('dump of cloned AST') refer to the same nodes, which must never happen. This means that their parent node doesn't implement clone() correctly."); @@ -1533,6 +1534,7 @@ private: std::cout << std::endl; WriteBufferFromOStream res_buf(std::cout, 4096); formatAST(*res, res_buf); + res_buf.next(); std::cout << std::endl << std::endl; } diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index e85e7a21261..87083c2c27b 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -86,7 +86,7 @@ Suggest::Suggest() void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeouts, size_t suggestion_limit) { - std::stringstream query; + std::stringstream query; // STYLE_CHECK_ALLOW_STD_STRING_STREAM query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" "SELECT name FROM system.functions" " UNION ALL " diff --git a/programs/client/TestHint.h b/programs/client/TestHint.h index 641c3e0ccf0..65666f4304c 100644 --- a/programs/client/TestHint.h +++ b/programs/client/TestHint.h @@ -93,7 +93,7 @@ private: void parse(const String & hint) { - std::stringstream ss; + std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM ss << hint; String item; diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 4ee14b14119..a129dc7efcc 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -162,7 +162,7 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts, if (!missing_partitions.empty()) { - std::stringstream ss; + WriteBufferFromOwnString ss; for (const String & missing_partition : missing_partitions) ss << " " << missing_partition; diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index 0f607ea5faf..ea2be469945 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -13,7 +13,7 @@ using ConfigurationPtr = Poco::AutoPtr; ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) { - std::stringstream ss(xml_data); + std::stringstream ss(xml_data); // STYLE_CHECK_ALLOW_STD_STRING_STREAM Poco::XML::InputSource input_source{ss}; return {new Poco::Util::XMLConfiguration{&input_source}}; } diff --git a/programs/copier/TaskTableAndShard.h b/programs/copier/TaskTableAndShard.h index cc12fe556c7..4f5bfb443e6 100644 --- a/programs/copier/TaskTableAndShard.h +++ b/programs/copier/TaskTableAndShard.h @@ -394,12 +394,8 @@ inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain() inline String DB::TaskShard::getDescription() const { - std::stringstream ss; - ss << "N" << numberInCluster() - << " (having a replica " << getHostNameExample() - << ", pull table " + getQuotedTable(task_table.table_pull) - << " of cluster " + task_table.cluster_pull_name << ")"; - return ss.str(); + return fmt::format("N{} (having a replica {}, pull table {} of cluster {}", + numberInCluster(), getHostNameExample(), getQuotedTable(task_table.table_pull), task_table.cluster_pull_name); } inline String DB::TaskShard::getHostNameExample() const diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 9ecc2a50a42..15e71198eb1 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -422,7 +422,7 @@ static const char * minimal_default_user_xml = static ConfigurationPtr getConfigurationFromXMLString(const char * xml_data) { - std::stringstream ss{std::string{xml_data}}; + std::stringstream ss{std::string{xml_data}}; // STYLE_CHECK_ALLOW_STD_STRING_STREAM Poco::XML::InputSource input_source{ss}; return {new Poco::Util::XMLConfiguration{&input_source}}; } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ed18793a537..ca445e83a14 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -191,10 +191,10 @@ int Server::run() if (config().hasOption("help")) { Poco::Util::HelpFormatter help_formatter(Server::options()); - std::stringstream header; - header << commandName() << " [OPTION] [-- [ARG]...]\n"; - header << "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010"; - help_formatter.setHeader(header.str()); + auto header_str = fmt::format("{} [OPTION] [-- [ARG]...]\n" + "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010", + commandName()); + help_formatter.setHeader(header_str); help_formatter.format(std::cout); return 0; } diff --git a/src/AggregateFunctions/AggregateFunctionRetention.h b/src/AggregateFunctions/AggregateFunctionRetention.h index b742dcdf77f..f8a2163ccb9 100644 --- a/src/AggregateFunctions/AggregateFunctionRetention.h +++ b/src/AggregateFunctions/AggregateFunctionRetention.h @@ -1,7 +1,5 @@ #pragma once -#include -#include #include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h b/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h index b755fbf081b..28c3f53d879 100644 --- a/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h +++ b/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h @@ -1,10 +1,8 @@ #pragma once #include -#include #include #include -#include #include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 3297819a9ff..2dd6ef8a9fd 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -1,7 +1,5 @@ #pragma once -#include -#include #include #include #include diff --git a/src/AggregateFunctions/ReservoirSampler.h b/src/AggregateFunctions/ReservoirSampler.h index c5d2158d59c..a321ef08648 100644 --- a/src/AggregateFunctions/ReservoirSampler.h +++ b/src/AggregateFunctions/ReservoirSampler.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/src/AggregateFunctions/ReservoirSamplerDeterministic.h index eae24c1f3e9..3b7817e9308 100644 --- a/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Columns/tests/gtest_weak_hash_32.cpp b/src/Columns/tests/gtest_weak_hash_32.cpp index a7fd60a3b9c..a04bd94124c 100644 --- a/src/Columns/tests/gtest_weak_hash_32.cpp +++ b/src/Columns/tests/gtest_weak_hash_32.cpp @@ -71,7 +71,7 @@ void checkColumn( std::unordered_map map; size_t num_collisions = 0; - std::stringstream collisions_str; + std::stringstream collisions_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM collisions_str.exceptions(std::ios::failbit); for (size_t i = 0; i < eq_class.size(); ++i) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 8a6093c47c9..3e06be94b11 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -17,6 +16,8 @@ #include #include #include +#include +#include #define PREPROCESSED_SUFFIX "-preprocessed" @@ -537,8 +538,7 @@ XMLDocumentPtr ConfigProcessor::processConfig( if (has_zk_includes) *has_zk_includes = !contributing_zk_paths.empty(); - std::stringstream comment; - comment.exceptions(std::ios::failbit); + WriteBufferFromOwnString comment; comment << " This file was generated automatically.\n"; comment << " Do not edit it: it is likely to be discarded and generated again before it's read next time.\n"; comment << " Files used to generate this file:"; diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 820e3857bfc..dd78d0ec9fc 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -245,8 +245,7 @@ static std::string getExtraExceptionInfo(const std::exception & e) std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace /*= false*/, bool with_extra_info /*= true*/) { - std::stringstream stream; - stream.exceptions(std::ios::failbit); + WriteBufferFromOwnString stream; try { @@ -365,8 +364,7 @@ void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::str std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace) { - std::stringstream stream; - stream.exceptions(std::ios::failbit); + WriteBufferFromOwnString stream; try { diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 5257f95898a..1d324a00473 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -133,17 +133,12 @@ void MemoryTracker::alloc(Int64 size) BlockerInThread untrack_lock; ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); - std::stringstream message; - message.exceptions(std::ios::failbit); - message << "Memory tracker"; - if (const auto * description = description_ptr.load(std::memory_order_relaxed)) - message << " " << description; - message << ": fault injected. Would use " << formatReadableSizeWithBinarySuffix(will_be) - << " (attempt to allocate chunk of " << size << " bytes)" - << ", maximum: " << formatReadableSizeWithBinarySuffix(current_hard_limit); - + const auto * description = description_ptr.load(std::memory_order_relaxed); amount.fetch_sub(size, std::memory_order_relaxed); - throw DB::Exception(message.str(), DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED); + throw DB::Exception(DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, + "Memory tracker {}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}", + description ? description : "", formatReadableSizeWithBinarySuffix(will_be), + size, formatReadableSizeWithBinarySuffix(current_hard_limit)); } if (unlikely(current_profiler_limit && will_be > current_profiler_limit)) @@ -166,17 +161,12 @@ void MemoryTracker::alloc(Int64 size) BlockerInThread untrack_lock; ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); - std::stringstream message; - message.exceptions(std::ios::failbit); - message << "Memory limit"; - if (const auto * description = description_ptr.load(std::memory_order_relaxed)) - message << " " << description; - message << " exceeded: would use " << formatReadableSizeWithBinarySuffix(will_be) - << " (attempt to allocate chunk of " << size << " bytes)" - << ", maximum: " << formatReadableSizeWithBinarySuffix(current_hard_limit); - + const auto * description = description_ptr.load(std::memory_order_relaxed); amount.fetch_sub(size, std::memory_order_relaxed); - throw DB::Exception(message.str(), DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED); + throw DB::Exception(DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, + "Memory limit {} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}", + description ? description : "", formatReadableSizeWithBinarySuffix(will_be), + size, formatReadableSizeWithBinarySuffix(current_hard_limit)); } updatePeak(will_be); diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index db0928ea605..069c7774729 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -73,8 +74,7 @@ ShellCommand::~ShellCommand() void ShellCommand::logCommand(const char * filename, char * const argv[]) { - std::stringstream args; - args.exceptions(std::ios::failbit); + WriteBufferFromOwnString args; for (int i = 0; argv != nullptr && argv[i] != nullptr; ++i) { if (i > 0) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 7e9474ac3a7..9cfdba2687f 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -23,7 +23,7 @@ std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context) { - std::stringstream error; + std::stringstream error; // STYLE_CHECK_ALLOW_STD_STRING_STREAM error.exceptions(std::ios::failbit); switch (sig) { @@ -319,7 +319,7 @@ static void toStringEveryLineImpl( const DB::SymbolIndex & symbol_index = DB::SymbolIndex::instance(); std::unordered_map dwarfs; - std::stringstream out; + std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM out.exceptions(std::ios::failbit); for (size_t i = offset; i < size; ++i) @@ -359,7 +359,7 @@ static void toStringEveryLineImpl( out.str({}); } #else - std::stringstream out; + std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM out.exceptions(std::ios::failbit); for (size_t i = offset; i < size; ++i) @@ -375,7 +375,7 @@ static void toStringEveryLineImpl( static std::string toStringImpl(const StackTrace::FramePointers & frame_pointers, size_t offset, size_t size) { - std::stringstream out; + std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM out.exceptions(std::ios::failbit); toStringEveryLineImpl(frame_pointers, offset, size, [&](const std::string & str) { out << str << '\n'; }); return out.str(); diff --git a/src/Common/StudentTTest.cpp b/src/Common/StudentTTest.cpp index fe605092acc..da4913126b3 100644 --- a/src/Common/StudentTTest.cpp +++ b/src/Common/StudentTTest.cpp @@ -153,7 +153,7 @@ std::pair StudentTTest::compareAndReport(size_t confidence_le double mean_confidence_interval = table_value * t_statistic; - std::stringstream ss; + std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM ss.exceptions(std::ios::failbit); if (mean_difference > mean_confidence_interval && (mean_difference - mean_confidence_interval > 0.0001)) /// difference must be more than 0.0001, to take into account connection latency. diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index f5ad28f57af..dad32293380 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -1,5 +1,3 @@ -#include - #include #include #include @@ -79,12 +77,10 @@ void ThreadStatus::assertState(const std::initializer_list & permitted_stat return; } - std::stringstream ss; - ss.exceptions(std::ios::failbit); - ss << "Unexpected thread state " << getCurrentState(); if (description) - ss << ": " << description; - throw Exception(ss.str(), ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}: {}", getCurrentState(), description); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}", getCurrentState()); } void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, diff --git a/src/Common/UInt128.h b/src/Common/UInt128.h index 735b287f90f..baa33578089 100644 --- a/src/Common/UInt128.h +++ b/src/Common/UInt128.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/src/Common/XDBCBridgeHelper.h b/src/Common/XDBCBridgeHelper.h index c794d2fe3cd..59bae33d88d 100644 --- a/src/Common/XDBCBridgeHelper.h +++ b/src/Common/XDBCBridgeHelper.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -307,9 +306,6 @@ struct ODBCBridgeMixin std::vector cmd_args; path.setFileName("clickhouse-odbc-bridge"); - std::stringstream command; - command.exceptions(std::ios::failbit); - #if !CLICKHOUSE_SPLIT_BINARY cmd_args.push_back("odbc-bridge"); #endif diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index f7db8a85f96..4ae26d874fb 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -218,7 +218,7 @@ std::pair TestKeeperCreateRequest::process(TestKeeper::Contai auto seq_num = it->second.seq_num; ++it->second.seq_num; - std::stringstream seq_num_str; + std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM seq_num_str.exceptions(std::ios::failbit); seq_num_str << std::setw(10) << std::setfill('0') << seq_num; diff --git a/src/Common/checkStackSize.cpp b/src/Common/checkStackSize.cpp index bdc117eccac..e94abc08c6b 100644 --- a/src/Common/checkStackSize.cpp +++ b/src/Common/checkStackSize.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #if defined(__FreeBSD__) # include @@ -80,13 +79,8 @@ __attribute__((__weak__)) void checkStackSize() /// It's safe to assume that overflow in multiplying by two cannot occur. if (stack_size * 2 > max_stack_size) { - std::stringstream message; - message.exceptions(std::ios::failbit); - message << "Stack size too large" - << ". Stack address: " << stack_address - << ", frame address: " << frame_address - << ", stack size: " << stack_size - << ", maximum stack size: " << max_stack_size; - throw Exception(message.str(), ErrorCodes::TOO_DEEP_RECURSION); + throw Exception(ErrorCodes::TOO_DEEP_RECURSION, + "Stack size too large. Stack address: {}, frame address: {}, stack size: {}, maximum stack size: {}", + stack_address, frame_address, stack_size, max_stack_size); } } diff --git a/src/Common/formatReadable.cpp b/src/Common/formatReadable.cpp index 99929275521..fc5c6c19b50 100644 --- a/src/Common/formatReadable.cpp +++ b/src/Common/formatReadable.cpp @@ -1,6 +1,4 @@ #include -#include -#include #include #include diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 7a6b605d015..8b16b68a999 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace ProfileEvents @@ -42,7 +43,7 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c if (expected_checksum == calculated_checksum) return; - std::stringstream message; + WriteBufferFromOwnString message; /// TODO mess up of endianness in error message. message << "Checksum doesn't match: corrupted data." @@ -50,7 +51,16 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c + ". Actual: " + getHexUIntLowercase(calculated_checksum.first) + getHexUIntLowercase(calculated_checksum.second) + ". Size of compressed block: " + toString(size); - const char * message_hardware_failure = "This is most likely due to hardware failure. If you receive broken data over network and the error does not repeat every time, this can be caused by bad RAM on network interface controller or bad controller itself or bad RAM on network switches or bad CPU on network switches (look at the logs on related network switches; note that TCP checksums don't help) or bad RAM on host (look at dmesg or kern.log for enormous amount of EDAC errors, ECC-related reports, Machine Check Exceptions, mcelog; note that ECC memory can fail if the number of errors is huge) or bad CPU on host. If you read data from disk, this can be caused by disk bit rott. This exception protects ClickHouse from data corruption due to hardware failures."; + const char * message_hardware_failure = "This is most likely due to hardware failure. " + "If you receive broken data over network and the error does not repeat every time, " + "this can be caused by bad RAM on network interface controller or bad controller itself " + "or bad RAM on network switches or bad CPU on network switches " + "(look at the logs on related network switches; note that TCP checksums don't help) " + "or bad RAM on host (look at dmesg or kern.log for enormous amount of EDAC errors, " + "ECC-related reports, Machine Check Exceptions, mcelog; note that ECC memory can fail " + "if the number of errors is huge) or bad CPU on host. If you read data from disk, " + "this can be caused by disk bit rott. This exception protects ClickHouse " + "from data corruption due to hardware failures."; auto flip_bit = [](char * buf, size_t pos) { diff --git a/src/Compression/tests/compressed_buffer.cpp b/src/Compression/tests/compressed_buffer.cpp index c018fc95995..aef2cf4ab90 100644 --- a/src/Compression/tests/compressed_buffer.cpp +++ b/src/Compression/tests/compressed_buffer.cpp @@ -51,10 +51,7 @@ int main(int, char **) if (x != i) { - std::stringstream s; - s.exceptions(std::ios::failbit); - s << "Failed!, read: " << x << ", expected: " << i; - throw DB::Exception(s.str(), 0); + throw DB::Exception(0, "Failed!, read: {}, expected: {}", x, i); } } stopwatch.stop(); diff --git a/src/Core/MySQL/IMySQLReadPacket.cpp b/src/Core/MySQL/IMySQLReadPacket.cpp index 676f3986ba4..74f309d0294 100644 --- a/src/Core/MySQL/IMySQLReadPacket.cpp +++ b/src/Core/MySQL/IMySQLReadPacket.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -21,10 +20,9 @@ void IMySQLReadPacket::readPayload(ReadBuffer & in, uint8_t & sequence_id) readPayloadImpl(payload); if (!payload.eof()) { - std::stringstream tmp; - tmp.exceptions(std::ios::failbit); - tmp << "Packet payload is not fully read. Stopped after " << payload.count() << " bytes, while " << payload.available() << " bytes are in buffer."; - throw Exception(tmp.str(), ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT); + throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT, + "Packet payload is not fully read. Stopped after {} bytes, while {} bytes are in buffer.", + payload.count(), payload.available()); } } diff --git a/src/Core/MySQL/IMySQLWritePacket.cpp b/src/Core/MySQL/IMySQLWritePacket.cpp index 3e97800177c..b5c95717a9b 100644 --- a/src/Core/MySQL/IMySQLWritePacket.cpp +++ b/src/Core/MySQL/IMySQLWritePacket.cpp @@ -1,10 +1,14 @@ #include #include -#include namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace MySQLProtocol { @@ -15,10 +19,8 @@ void IMySQLWritePacket::writePayload(WriteBuffer & buffer, uint8_t & sequence_id buf.next(); if (buf.remainingPayloadSize()) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - ss << "Incomplete payload. Written " << getPayloadSize() - buf.remainingPayloadSize() << " bytes, expected " << getPayloadSize() << " bytes."; - throw Exception(ss.str(), 0); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incomplete payload. Written {} bytes, expected {} bytes.", + getPayloadSize() - buf.remainingPayloadSize(), getPayloadSize()); } } diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index c09c4b3b034..6ff1670777a 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -35,15 +36,15 @@ namespace MySQLReplication payload.readStrict(reinterpret_cast(&flags), 2); } - void EventHeader::dump(std::ostream & out) const + void EventHeader::dump(WriteBuffer & out) const { - out << "\n=== " << to_string(this->type) << " ===" << std::endl; - out << "Timestamp: " << this->timestamp << std::endl; - out << "Event Type: " << this->type << std::endl; - out << "Server ID: " << this->server_id << std::endl; - out << "Event Size: " << this->event_size << std::endl; - out << "Log Pos: " << this->log_pos << std::endl; - out << "Flags: " << this->flags << std::endl; + out << "\n=== " << to_string(this->type) << " ===" << '\n'; + out << "Timestamp: " << this->timestamp << '\n'; + out << "Event Type: " << to_string(this->type) << '\n'; + out << "Server ID: " << this->server_id << '\n'; + out << "Event Size: " << this->event_size << '\n'; + out << "Log Pos: " << this->log_pos << '\n'; + out << "Flags: " << this->flags << '\n'; } /// https://dev.mysql.com/doc/internals/en/format-description-event.html @@ -60,13 +61,13 @@ namespace MySQLReplication readStringUntilEOF(event_type_header_length, payload); } - void FormatDescriptionEvent::dump(std::ostream & out) const + void FormatDescriptionEvent::dump(WriteBuffer & out) const { header.dump(out); - out << "Binlog Version: " << this->binlog_version << std::endl; - out << "Server Version: " << this->server_version << std::endl; - out << "Create Timestamp: " << this->create_timestamp << std::endl; - out << "Event Header Len: " << std::to_string(this->event_header_length) << std::endl; + out << "Binlog Version: " << this->binlog_version << '\n'; + out << "Server Version: " << this->server_version << '\n'; + out << "Create Timestamp: " << this->create_timestamp << '\n'; + out << "Event Header Len: " << std::to_string(this->event_header_length) << '\n'; } /// https://dev.mysql.com/doc/internals/en/rotate-event.html @@ -76,11 +77,11 @@ namespace MySQLReplication readStringUntilEOF(next_binlog, payload); } - void RotateEvent::dump(std::ostream & out) const + void RotateEvent::dump(WriteBuffer & out) const { header.dump(out); - out << "Position: " << this->position << std::endl; - out << "Next Binlog: " << this->next_binlog << std::endl; + out << "Position: " << this->position << '\n'; + out << "Next Binlog: " << this->next_binlog << '\n'; } /// https://dev.mysql.com/doc/internals/en/query-event.html @@ -116,24 +117,24 @@ namespace MySQLReplication } } - void QueryEvent::dump(std::ostream & out) const + void QueryEvent::dump(WriteBuffer & out) const { header.dump(out); - out << "Thread ID: " << this->thread_id << std::endl; - out << "Execution Time: " << this->exec_time << std::endl; - out << "Schema Len: " << std::to_string(this->schema_len) << std::endl; - out << "Error Code: " << this->error_code << std::endl; - out << "Status Len: " << this->status_len << std::endl; - out << "Schema: " << this->schema << std::endl; - out << "Query: " << this->query << std::endl; + out << "Thread ID: " << this->thread_id << '\n'; + out << "Execution Time: " << this->exec_time << '\n'; + out << "Schema Len: " << std::to_string(this->schema_len) << '\n'; + out << "Error Code: " << this->error_code << '\n'; + out << "Status Len: " << this->status_len << '\n'; + out << "Schema: " << this->schema << '\n'; + out << "Query: " << this->query << '\n'; } void XIDEvent::parseImpl(ReadBuffer & payload) { payload.readStrict(reinterpret_cast(&xid), 8); } - void XIDEvent::dump(std::ostream & out) const + void XIDEvent::dump(WriteBuffer & out) const { header.dump(out); - out << "XID: " << this->xid << std::endl; + out << "XID: " << this->xid << '\n'; } void TableMapEvent::parseImpl(ReadBuffer & payload) @@ -238,21 +239,23 @@ namespace MySQLReplication } } - void TableMapEvent::dump(std::ostream & out) const + void TableMapEvent::dump(WriteBuffer & out) const { header.dump(out); - out << "Table ID: " << this->table_id << std::endl; - out << "Flags: " << this->flags << std::endl; - out << "Schema Len: " << std::to_string(this->schema_len) << std::endl; - out << "Schema: " << this->schema << std::endl; - out << "Table Len: " << std::to_string(this->table_len) << std::endl; - out << "Table: " << this->table << std::endl; - out << "Column Count: " << this->column_count << std::endl; + out << "Table ID: " << this->table_id << '\n'; + out << "Flags: " << this->flags << '\n'; + out << "Schema Len: " << std::to_string(this->schema_len) << '\n'; + out << "Schema: " << this->schema << '\n'; + out << "Table Len: " << std::to_string(this->table_len) << '\n'; + out << "Table: " << this->table << '\n'; + out << "Column Count: " << this->column_count << '\n'; for (auto i = 0U; i < column_count; i++) { - out << "Column Type [" << i << "]: " << std::to_string(column_type[i]) << ", Meta: " << column_meta[i] << std::endl; + out << "Column Type [" << i << "]: " << std::to_string(column_type[i]) << ", Meta: " << column_meta[i] << '\n'; } - out << "Null Bitmap: " << this->null_bitmap << std::endl; + String bitmap_str; + boost::to_string(this->null_bitmap, bitmap_str); + out << "Null Bitmap: " << bitmap_str << '\n'; } void RowsEvent::parseImpl(ReadBuffer & payload) @@ -631,16 +634,16 @@ namespace MySQLReplication rows.push_back(row); } - void RowsEvent::dump(std::ostream & out) const + void RowsEvent::dump(WriteBuffer & out) const { FieldVisitorToString to_string; header.dump(out); - out << "Schema: " << this->schema << std::endl; - out << "Table: " << this->table << std::endl; + out << "Schema: " << this->schema << '\n'; + out << "Table: " << this->table << '\n'; for (auto i = 0U; i < rows.size(); i++) { - out << "Row[" << i << "]: " << applyVisitor(to_string, rows[i]) << std::endl; + out << "Row[" << i << "]: " << applyVisitor(to_string, rows[i]) << '\n'; } } @@ -663,22 +666,22 @@ namespace MySQLReplication payload.ignoreAll(); } - void GTIDEvent::dump(std::ostream & out) const + void GTIDEvent::dump(WriteBuffer & out) const { WriteBufferFromOwnString ws; writeUUIDText(gtid.uuid, ws); auto gtid_next = ws.str() + ":" + std::to_string(gtid.seq_no); header.dump(out); - out << "GTID Next: " << gtid_next << std::endl; + out << "GTID Next: " << gtid_next << '\n'; } void DryRunEvent::parseImpl(ReadBuffer & payload) { payload.ignoreAll(); } - void DryRunEvent::dump(std::ostream & out) const + void DryRunEvent::dump(WriteBuffer & out) const { header.dump(out); - out << "[DryRun Event]" << std::endl; + out << "[DryRun Event]" << '\n'; } /// Update binlog name/position/gtid based on the event type. @@ -716,12 +719,12 @@ namespace MySQLReplication gtid_sets.parse(gtid_sets_); } - void Position::dump(std::ostream & out) const + void Position::dump(WriteBuffer & out) const { - out << "\n=== Binlog Position ===" << std::endl; - out << "Binlog: " << this->binlog_name << std::endl; - out << "Position: " << this->binlog_pos << std::endl; - out << "GTIDSets: " << this->gtid_sets.toString() << std::endl; + out << "\n=== Binlog Position ===" << '\n'; + out << "Binlog: " << this->binlog_name << '\n'; + out << "Position: " << this->binlog_pos << '\n'; + out << "GTIDSets: " << this->gtid_sets.toString() << '\n'; } void MySQLFlavor::readPayloadImpl(ReadBuffer & payload) diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 6f5b4cf0a1e..394ac729d1b 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -309,7 +309,7 @@ namespace MySQLReplication UInt16 flags; EventHeader() : timestamp(0), server_id(0), event_size(0), log_pos(0), flags(0) { } - void dump(std::ostream & out) const; + void dump(WriteBuffer & out) const; void parse(ReadBuffer & payload); }; @@ -321,7 +321,7 @@ namespace MySQLReplication EventBase(EventHeader && header_) : header(std::move(header_)) {} virtual ~EventBase() = default; - virtual void dump(std::ostream & out) const = 0; + virtual void dump(WriteBuffer & out) const = 0; virtual void parseEvent(ReadBuffer & payload) { parseImpl(payload); } virtual MySQLEventType type() const { return MYSQL_UNHANDLED_EVENT; } @@ -344,7 +344,7 @@ namespace MySQLReplication UInt8 event_header_length; String event_type_header_length; - void dump(std::ostream & out) const override; + void dump(WriteBuffer & out) const override; void parseImpl(ReadBuffer & payload) override; private: @@ -358,7 +358,7 @@ namespace MySQLReplication String next_binlog; RotateEvent(EventHeader && header_) : EventBase(std::move(header_)), position(0) {} - void dump(std::ostream & out) const override; + void dump(WriteBuffer & out) const override; protected: void parseImpl(ReadBuffer & payload) override; @@ -389,7 +389,7 @@ namespace MySQLReplication { } - void dump(std::ostream & out) const override; + void dump(WriteBuffer & out) const override; MySQLEventType type() const override { return MYSQL_QUERY_EVENT; } protected: @@ -404,7 +404,7 @@ namespace MySQLReplication protected: UInt64 xid; - void dump(std::ostream & out) const override; + void dump(WriteBuffer & out) const override; void parseImpl(ReadBuffer & payload) override; }; @@ -423,7 +423,7 @@ namespace MySQLReplication Bitmap null_bitmap; TableMapEvent(EventHeader && header_) : EventBase(std::move(header_)), table_id(0), flags(0), schema_len(0), table_len(0), column_count(0) {} - void dump(std::ostream & out) const override; + void dump(WriteBuffer & out) const override; protected: void parseImpl(ReadBuffer & payload) override; @@ -445,7 +445,7 @@ namespace MySQLReplication table = table_map->table; } - void dump(std::ostream & out) const override; + void dump(WriteBuffer & out) const override; protected: UInt64 table_id; @@ -489,7 +489,7 @@ namespace MySQLReplication GTID gtid; GTIDEvent(EventHeader && header_) : EventBase(std::move(header_)), commit_flag(0) {} - void dump(std::ostream & out) const override; + void dump(WriteBuffer & out) const override; protected: void parseImpl(ReadBuffer & payload) override; @@ -499,7 +499,7 @@ namespace MySQLReplication { public: DryRunEvent(EventHeader && header_) : EventBase(std::move(header_)) {} - void dump(std::ostream & out) const override; + void dump(WriteBuffer & out) const override; protected: void parseImpl(ReadBuffer & payload) override; @@ -515,7 +515,7 @@ namespace MySQLReplication Position() : binlog_pos(0) { } void update(BinlogEventPtr event); void update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_); - void dump(std::ostream & out) const; + void dump(WriteBuffer & out) const; }; class IFlavor : public MySQLProtocol::IMySQLReadPacket diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index 98229bb73d7..79ac7ddf142 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -60,10 +60,7 @@ struct SortColumnDescription std::string dump() const { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - ss << column_name << ":" << column_number << ":dir " << direction << "nulls " << nulls_direction; - return ss.str(); + return fmt::format("{}:{}:dir {}nulls ", column_name, column_number, direction, nulls_direction); } }; diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index 7e6aae5da23..9dc46891241 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -329,6 +330,8 @@ int main(int argc, char ** argv) slave.connect(); slave.startBinlogDumpGTID(slave_id, replicate_db, gtid_sets); + WriteBufferFromOStream cerr(std::cerr); + /// Read one binlog event on by one. while (true) { @@ -337,40 +340,40 @@ int main(int argc, char ** argv) { case MYSQL_QUERY_EVENT: { auto binlog_event = std::static_pointer_cast(event); - binlog_event->dump(std::cerr); + binlog_event->dump(cerr); Position pos = slave.getPosition(); - pos.dump(std::cerr); + pos.dump(cerr); break; } case MYSQL_WRITE_ROWS_EVENT: { auto binlog_event = std::static_pointer_cast(event); - binlog_event->dump(std::cerr); + binlog_event->dump(cerr); Position pos = slave.getPosition(); - pos.dump(std::cerr); + pos.dump(cerr); break; } case MYSQL_UPDATE_ROWS_EVENT: { auto binlog_event = std::static_pointer_cast(event); - binlog_event->dump(std::cerr); + binlog_event->dump(cerr); Position pos = slave.getPosition(); - pos.dump(std::cerr); + pos.dump(cerr); break; } case MYSQL_DELETE_ROWS_EVENT: { auto binlog_event = std::static_pointer_cast(event); - binlog_event->dump(std::cerr); + binlog_event->dump(cerr); Position pos = slave.getPosition(); - pos.dump(std::cerr); + pos.dump(cerr); break; } default: if (event->header.type != MySQLReplication::EventType::HEARTBEAT_EVENT) { - event->dump(std::cerr); + event->dump(cerr); } break; } diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index d47d82689de..d4a892711ef 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -59,15 +59,10 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) /// Is violated. if (!value) { - std::stringstream exception_message; - exception_message.exceptions(std::ios::failbit); - - exception_message << "Constraint " << backQuote(constraint_ptr->name) - << " for table " << table_id.getNameForLogs() - << " is violated, because it is a constant expression returning 0." - << " It is most likely an error in table definition."; - - throw Exception{exception_message.str(), ErrorCodes::VIOLATED_CONSTRAINT}; + throw Exception(ErrorCodes::VIOLATED_CONSTRAINT, + "Constraint {} for table {} is violated, because it is a constant expression returning 0. " + "It is most likely an error in table definition.", + backQuote(constraint_ptr->name), table_id.getNameForLogs()); } } else @@ -87,28 +82,27 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) Names related_columns = constraint_expr->getRequiredColumns(); - std::stringstream exception_message; - exception_message.exceptions(std::ios::failbit); - - exception_message << "Constraint " << backQuote(constraint_ptr->name) - << " for table " << table_id.getNameForLogs() - << " is violated at row " << (rows_written + row_idx + 1) - << ". Expression: (" << serializeAST(*(constraint_ptr->expr), true) << ")" - << ". Column values"; - bool first = true; + String column_values_msg; + constexpr size_t approx_bytes_for_col = 32; + column_values_msg.reserve(approx_bytes_for_col * related_columns.size()); for (const auto & name : related_columns) { const IColumn & column = *block.getByName(name).column; assert(row_idx < column.size()); - exception_message << (first ? ": " : ", ") - << backQuoteIfNeed(name) << " = " << applyVisitor(FieldVisitorToString(), column[row_idx]); - + if (!first) + column_values_msg.append(", "); + column_values_msg.append(backQuoteIfNeed(name)); + column_values_msg.append(" = "); + column_values_msg.append(applyVisitor(FieldVisitorToString(), column[row_idx])); first = false; } - throw Exception{exception_message.str(), ErrorCodes::VIOLATED_CONSTRAINT}; + throw Exception(ErrorCodes::VIOLATED_CONSTRAINT, + "Constraint {} for table {} is violated at row {}. Expression: ({}). Column values: {}", + backQuote(constraint_ptr->name), table_id.getNameForLogs(), rows_written + row_idx + 1, + serializeAST(*(constraint_ptr->expr), true), column_values_msg); } } } diff --git a/src/DataStreams/IBlockInputStream.cpp b/src/DataStreams/IBlockInputStream.cpp index 23ba9ff2970..a6484c41b4f 100644 --- a/src/DataStreams/IBlockInputStream.cpp +++ b/src/DataStreams/IBlockInputStream.cpp @@ -4,7 +4,8 @@ #include #include #include -#include +#include +#include namespace ProfileEvents { @@ -359,8 +360,7 @@ Block IBlockInputStream::getExtremes() String IBlockInputStream::getTreeID() const { - std::stringstream s; - s.exceptions(std::ios::failbit); + WriteBufferFromOwnString s; s << getName(); if (!children.empty()) @@ -399,13 +399,13 @@ size_t IBlockInputStream::checkDepthImpl(size_t max_depth, size_t level) const } -void IBlockInputStream::dumpTree(std::ostream & ostr, size_t indent, size_t multiplier) const +void IBlockInputStream::dumpTree(WriteBuffer & ostr, size_t indent, size_t multiplier) const { ostr << String(indent, ' ') << getName(); if (multiplier > 1) ostr << " × " << multiplier; //ostr << ": " << getHeader().dumpStructure(); - ostr << std::endl; + ostr << '\n'; ++indent; /// If the subtree is repeated several times, then we output it once with the multiplier. diff --git a/src/DataStreams/IBlockInputStream.h b/src/DataStreams/IBlockInputStream.h index 3fbc3ce4bcd..4e314ef2980 100644 --- a/src/DataStreams/IBlockInputStream.h +++ b/src/DataStreams/IBlockInputStream.h @@ -95,7 +95,7 @@ public: virtual void readSuffix(); /// Must be called before `read()` and `readPrefix()`. - void dumpTree(std::ostream & ostr, size_t indent = 0, size_t multiplier = 1) const; + void dumpTree(WriteBuffer & ostr, size_t indent = 0, size_t multiplier = 1) const; /** Check the depth of the pipeline. * If max_depth is specified and the `depth` is greater - throw an exception. diff --git a/src/DataStreams/MergingSortedBlockInputStream.cpp b/src/DataStreams/MergingSortedBlockInputStream.cpp index 9c9213ef3cc..ee6b93dc8e4 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -1,6 +1,4 @@ #include -#include -#include #include diff --git a/src/DataStreams/MongoDBBlockInputStream.cpp b/src/DataStreams/MongoDBBlockInputStream.cpp index 25abdd909c4..5463d95151b 100644 --- a/src/DataStreams/MongoDBBlockInputStream.cpp +++ b/src/DataStreams/MongoDBBlockInputStream.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp index 3fe17a4bbfc..1d063dac697 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp @@ -32,8 +32,7 @@ static const std::vector supported_functions{"any", "anyLast", "min", String DataTypeCustomSimpleAggregateFunction::getName() const { - std::stringstream stream; - stream.exceptions(std::ios::failbit); + WriteBufferFromOwnString stream; stream << "SimpleAggregateFunction(" << function->getName(); if (!parameters.empty()) diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index bd4329f6f58..6c325c5d371 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -29,10 +29,7 @@ namespace ErrorCodes template std::string DataTypeDecimal::doGetName() const { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - ss << "Decimal(" << this->precision << ", " << this->scale << ")"; - return ss.str(); + return fmt::format("Decimal({}, {})", this->precision, this->scale); } diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 2a3de25c24f..7e42b2548b0 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -127,8 +127,7 @@ static String checkVariableAndGetVersion(const mysqlxx::Pool::Entry & connection } bool first = true; - std::stringstream error_message; - error_message.exceptions(std::ios::failbit); + WriteBufferFromOwnString error_message; error_message << "Illegal MySQL variables, the MaterializeMySQL engine requires "; for (const auto & [variable_name, variable_error_message] : variables_error_message) { @@ -239,8 +238,7 @@ static inline BlockOutputStreamPtr getTableOutput(const String & database_name, { const StoragePtr & storage = DatabaseCatalog::instance().getTable(StorageID(database_name, table_name), query_context); - std::stringstream insert_columns_str; - insert_columns_str.exceptions(std::ios::failbit); + WriteBufferFromOwnString insert_columns_str; const StorageInMemoryMetadata & storage_metadata = storage->getInMemoryMetadata(); const ColumnsDescription & storage_columns = storage_metadata.getColumns(); const NamesAndTypesList & insert_columns_names = insert_materialized ? storage_columns.getAllPhysical() : storage_columns.getOrdinary(); @@ -331,10 +329,9 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz const auto & position_message = [&]() { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - position.dump(ss); - return ss.str(); + WriteBufferFromOwnString buf; + position.dump(buf); + return buf.str(); }; LOG_INFO(log, "MySQL dump database position: \n {}", position_message()); } @@ -374,10 +371,9 @@ void MaterializeMySQLSyncThread::flushBuffersData(Buffers & buffers, Materialize const auto & position_message = [&]() { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - client.getPosition().dump(ss); - return ss.str(); + WriteBufferFromOwnString buf; + client.getPosition().dump(buf); + return buf.str(); }; LOG_INFO(log, "MySQL executed position: \n {}", position_message()); } @@ -632,10 +628,9 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr { const auto & dump_event_message = [&]() { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - receive_event->dump(ss); - return ss.str(); + WriteBufferFromOwnString buf; + receive_event->dump(buf); + return buf.str(); }; LOG_DEBUG(log, "Skip MySQL event: \n {}", dump_event_message()); diff --git a/src/Functions/abtesting.cpp b/src/Functions/abtesting.cpp index 051b1f6f0ef..9f919051ee9 100644 --- a/src/Functions/abtesting.cpp +++ b/src/Functions/abtesting.cpp @@ -3,7 +3,6 @@ #if !defined(ARCADIA_BUILD) && USE_STATS #include -#include #include #include diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 04fec145775..fa30afbd54f 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -239,7 +239,7 @@ void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPR if (!(status == Poco::Net::HTTPResponse::HTTP_OK || (isRedirect(status) && allow_redirects))) { - std::stringstream error_message; + std::stringstream error_message; // STYLE_CHECK_ALLOW_STD_STRING_STREAM error_message.exceptions(std::ios::failbit); error_message << "Received error from remote server " << request.getURI() << ". HTTP status code: " << status << " " << response.getReason() << ", body: " << istr.rdbuf(); diff --git a/src/IO/MySQLPacketPayloadReadBuffer.cpp b/src/IO/MySQLPacketPayloadReadBuffer.cpp index f6f899e0ac7..9ca7845b2ae 100644 --- a/src/IO/MySQLPacketPayloadReadBuffer.cpp +++ b/src/IO/MySQLPacketPayloadReadBuffer.cpp @@ -1,5 +1,4 @@ #include -#include namespace DB { diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 49ccb6dc1b3..4a5b79e31ea 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include #include @@ -247,8 +249,7 @@ void PocoHTTPClient::makeRequestInternal( response->SetResponseCode(static_cast(status_code)); response->SetContentType(poco_response.getContentType()); - std::stringstream headers_ss; - headers_ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString headers_ss; for (const auto & [header_name, header_value] : poco_response) { response->AddHeader(header_name, header_value); diff --git a/src/IO/tests/gtest_bit_io.cpp b/src/IO/tests/gtest_bit_io.cpp index dce146eaad7..2302a6c6135 100644 --- a/src/IO/tests/gtest_bit_io.cpp +++ b/src/IO/tests/gtest_bit_io.cpp @@ -76,7 +76,7 @@ std::string dumpContents(const T& container, const size_t cols_in_row = 8) { - std::stringstream sstr; + std::stringstream sstr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM sstr.exceptions(std::ios::failbit); dumpBuffer(std::begin(container), std::end(container), &sstr, col_sep, row_sep, cols_in_row); diff --git a/src/IO/tests/hashing_read_buffer.cpp b/src/IO/tests/hashing_read_buffer.cpp index a1140160c32..67b3d61b1dc 100644 --- a/src/IO/tests/hashing_read_buffer.cpp +++ b/src/IO/tests/hashing_read_buffer.cpp @@ -22,7 +22,7 @@ static void test(size_t data_size) for (size_t read_buffer_block_size : block_sizes) { std::cout << "block size " << read_buffer_block_size << std::endl; - std::stringstream io; + std::stringstream io; // STYLE_CHECK_ALLOW_STD_STRING_STREAM io.exceptions(std::ios::failbit); DB::WriteBufferFromOStream out_impl(io); DB::HashingWriteBuffer out(out_impl); diff --git a/src/IO/tests/limit_read_buffer2.cpp b/src/IO/tests/limit_read_buffer2.cpp index 416eae0966b..d70ad832020 100644 --- a/src/IO/tests/limit_read_buffer2.cpp +++ b/src/IO/tests/limit_read_buffer2.cpp @@ -20,7 +20,7 @@ try { using namespace DB; - std::stringstream s; + std::stringstream s; // STYLE_CHECK_ALLOW_STD_STRING_STREAM s.exceptions(std::ios::failbit); { diff --git a/src/IO/tests/write_buffer.cpp b/src/IO/tests/write_buffer.cpp index c0e9150d372..09b46f810bf 100644 --- a/src/IO/tests/write_buffer.cpp +++ b/src/IO/tests/write_buffer.cpp @@ -16,7 +16,7 @@ int main(int, char **) DB::String c = "вася пе\tтя"; DB::String d = "'xyz\\"; - std::stringstream s; + std::stringstream s; // STYLE_CHECK_ALLOW_STD_STRING_STREAM s.exceptions(std::ios::failbit); { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a22eacd9a49..75e586ec34f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1965,21 +1965,19 @@ void Context::checkCanBeDropped(const String & database, const String & table, c String size_str = formatReadableSizeWithDecimalSuffix(size); String max_size_to_drop_str = formatReadableSizeWithDecimalSuffix(max_size_to_drop); - std::stringstream ostr; - ostr.exceptions(std::ios::failbit); - - ostr << "Table or Partition in " << backQuoteIfNeed(database) << "." << backQuoteIfNeed(table) << " was not dropped.\n" - << "Reason:\n" - << "1. Size (" << size_str << ") is greater than max_[table/partition]_size_to_drop (" << max_size_to_drop_str << ")\n" - << "2. File '" << force_file.path() << "' intended to force DROP " - << (force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist") << "\n"; - - ostr << "How to fix this:\n" - << "1. Either increase (or set to zero) max_[table/partition]_size_to_drop in server config\n" - << "2. Either create forcing file " << force_file.path() << " and make sure that ClickHouse has write permission for it.\n" - << "Example:\nsudo touch '" << force_file.path() << "' && sudo chmod 666 '" << force_file.path() << "'"; - - throw Exception(ostr.str(), ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT); + throw Exception(ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT, + "Table or Partition in {}.{} was not dropped.\nReason:\n" + "1. Size ({}) is greater than max_[table/partition]_size_to_drop ({})\n" + "2. File '{}' intended to force DROP {}\n" + "How to fix this:\n" + "1. Either increase (or set to zero) max_[table/partition]_size_to_drop in server config\n", + "2. Either create forcing file {} and make sure that ClickHouse has write permission for it.\n" + "Example:\nsudo touch '{}' && sudo chmod 666 '{}'", + backQuoteIfNeed(database), backQuoteIfNeed(table), + size_str, max_size_to_drop_str, + force_file.path(), force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist", + force_file.path(), + force_file.path(), force_file.path()); } diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 75872553ec3..582937c6ac1 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -453,8 +454,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const std::string ExpressionAction::toString() const { - std::stringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString ss; switch (type) { case ADD_COLUMN: @@ -550,8 +550,7 @@ void ExpressionActions::checkLimits(Block & block) const if (non_const_columns > settings.max_temporary_non_const_columns) { - std::stringstream list_of_non_const_columns; - list_of_non_const_columns.exceptions(std::ios::failbit); + WriteBufferFromOwnString list_of_non_const_columns; for (size_t i = 0, size = block.columns(); i < size; ++i) if (block.safeGetByPosition(i).column && !isColumnConst(*block.safeGetByPosition(i).column)) list_of_non_const_columns << "\n" << block.safeGetByPosition(i).name; @@ -922,8 +921,7 @@ void ExpressionActions::finalize(const Names & output_columns) std::string ExpressionActions::dumpActions() const { - std::stringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString ss; ss << "input:\n"; for (const auto & input_column : input_columns) @@ -1344,8 +1342,7 @@ void ExpressionActionsChain::finalize() std::string ExpressionActionsChain::dumpChain() const { - std::stringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString ss; for (size_t i = 0; i < steps.size(); ++i) { diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index ae8e2637cbb..b13350d7ba2 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -11,10 +11,8 @@ #include #include #include -#include #include -#include #include #include diff --git a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp index a81245adfc9..3135b0cfdf2 100644 --- a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp @@ -26,7 +26,6 @@ #include #include #include -#include namespace DB diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 32f461863c9..907523ce94b 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -1,5 +1,3 @@ -#include - #include #include #include diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 8e67cecdd5e..861fa8c3505 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -6,8 +6,7 @@ #include #include #include -#include -#include +#include namespace DB @@ -32,8 +31,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() /// SHOW DATABASES if (query.databases) { - std::stringstream rewritten_query; - rewritten_query.exceptions(std::ios::failbit); + WriteBufferFromOwnString rewritten_query; rewritten_query << "SELECT name FROM system.databases"; if (!query.like.empty()) @@ -42,7 +40,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() << " WHERE name " << (query.not_like ? "NOT " : "") << (query.case_insensitive_like ? "ILIKE " : "LIKE ") - << std::quoted(query.like, '\''); + << DB::quote << query.like; } if (query.limit_length) @@ -54,8 +52,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() /// SHOW CLUSTER/CLUSTERS if (query.clusters) { - std::stringstream rewritten_query; - rewritten_query.exceptions(std::ios::failbit); + WriteBufferFromOwnString rewritten_query; rewritten_query << "SELECT DISTINCT cluster FROM system.clusters"; if (!query.like.empty()) @@ -64,7 +61,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() << " WHERE cluster " << (query.not_like ? "NOT " : "") << (query.case_insensitive_like ? "ILIKE " : "LIKE ") - << std::quoted(query.like, '\''); + << DB::quote << query.like; } if (query.limit_length) @@ -74,11 +71,10 @@ String InterpreterShowTablesQuery::getRewrittenQuery() } else if (query.cluster) { - std::stringstream rewritten_query; - rewritten_query.exceptions(std::ios::failbit); + WriteBufferFromOwnString rewritten_query; rewritten_query << "SELECT * FROM system.clusters"; - rewritten_query << " WHERE cluster = " << std::quoted(query.cluster_str, '\''); + rewritten_query << " WHERE cluster = " << DB::quote << query.cluster_str; return rewritten_query.str(); } @@ -89,8 +85,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() String database = context.resolveDatabase(query.from); DatabaseCatalog::instance().assertDatabaseExists(database); - std::stringstream rewritten_query; - rewritten_query.exceptions(std::ios::failbit); + WriteBufferFromOwnString rewritten_query; rewritten_query << "SELECT name FROM system."; if (query.dictionaries) @@ -107,14 +102,14 @@ String InterpreterShowTablesQuery::getRewrittenQuery() rewritten_query << "is_temporary"; } else - rewritten_query << "database = " << std::quoted(database, '\''); + rewritten_query << "database = " << DB::quote << database; if (!query.like.empty()) rewritten_query << " AND name " << (query.not_like ? "NOT " : "") << (query.case_insensitive_like ? "ILIKE " : "LIKE ") - << std::quoted(query.like, '\''); + << DB::quote << query.like; else if (query.where_expression) rewritten_query << " AND (" << query.where_expression << ")"; diff --git a/src/Interpreters/QueryAliasesVisitor.cpp b/src/Interpreters/QueryAliasesVisitor.cpp index 00c337920f4..d395bfc20e9 100644 --- a/src/Interpreters/QueryAliasesVisitor.cpp +++ b/src/Interpreters/QueryAliasesVisitor.cpp @@ -1,6 +1,3 @@ -#include -#include - #include #include #include diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 907cbaee243..8eeb939329b 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -342,11 +342,9 @@ void Set::checkColumnsNumber(size_t num_key_columns) const { if (data_types.size() != num_key_columns) { - std::stringstream message; - message.exceptions(std::ios::failbit); - message << "Number of columns in section IN doesn't match. " - << num_key_columns << " at left, " << data_types.size() << " at right."; - throw Exception(message.str(), ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, + "Number of columns in section IN doesn't match. {} at left, {} at right.", + num_key_columns, data_types.size()); } } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 8d3cb123955..f6ff33112d0 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -552,8 +552,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (!unknown_required_source_columns.empty()) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString ss; ss << "Missing columns:"; for (const auto & name : unknown_required_source_columns) ss << " '" << name << "'"; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cdb3d9b7d7b..9de788a94d4 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -778,11 +778,9 @@ static std::tuple executeQueryImpl( if (!internal && res.in) { - std::stringstream log_str; - log_str.exceptions(std::ios::failbit); - log_str << "Query pipeline:\n"; - res.in->dumpTree(log_str); - LOG_DEBUG(&Poco::Logger::get("executeQuery"), log_str.str()); + WriteBufferFromOwnString msg_buf; + res.in->dumpTree(msg_buf); + LOG_DEBUG(&Poco::Logger::get("executeQuery"), "Query pipeline:\n{}", msg_buf.str()); } } } diff --git a/src/Parsers/ASTCreateRowPolicyQuery.cpp b/src/Parsers/ASTCreateRowPolicyQuery.cpp index 241d3ff051a..30b001feeca 100644 --- a/src/Parsers/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/ASTCreateRowPolicyQuery.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include diff --git a/src/Parsers/ASTIdentifier.cpp b/src/Parsers/ASTIdentifier.cpp index d51f37a0047..2717edd3b91 100644 --- a/src/Parsers/ASTIdentifier.cpp +++ b/src/Parsers/ASTIdentifier.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include diff --git a/src/Parsers/ASTWithAlias.cpp b/src/Parsers/ASTWithAlias.cpp index 1feb89f4bdc..20f647fb575 100644 --- a/src/Parsers/ASTWithAlias.cpp +++ b/src/Parsers/ASTWithAlias.cpp @@ -1,5 +1,4 @@ #include -#include #include #include diff --git a/src/Parsers/DumpASTNode.h b/src/Parsers/DumpASTNode.h index 1208aeca2a9..27ea3b50d62 100644 --- a/src/Parsers/DumpASTNode.h +++ b/src/Parsers/DumpASTNode.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 2a74c484187..bc00e2afe77 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -1,4 +1,3 @@ -#include #include #include #include diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 9428f312106..01ce4971c45 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -8,9 +8,7 @@ #include #include -#include #include -#include class SipHash; diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index 3a258df099e..fca8ea0aa35 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -1,6 +1,5 @@ #include -#include namespace DB { diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index bee89521812..15381b62028 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -16,16 +16,16 @@ void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one String serializeAST(const IAST & ast, bool one_line = true); -//inline std::ostream & operator<<(std::ostream & os, const IAST & ast) -//{ -// formatAST(ast, os, false, true); -// return os; -//} -// -//inline std::ostream & operator<<(std::ostream & os, const ASTPtr & ast) -//{ -// formatAST(*ast, os, false, true); -// return os; -//} +inline WriteBuffer & operator<<(WriteBuffer & buf, const IAST & ast) +{ + formatAST(ast, buf, false, true); + return buf; +} + +inline WriteBuffer & operator<<(WriteBuffer & buf, const ASTPtr & ast) +{ + formatAST(*ast, buf, false, true); + return buf; +} } diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 067c5cb0ab9..f40261b4561 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -2,8 +2,6 @@ #include #include #include -#include -#include namespace DB { diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 96a458eb49f..8bd4d36532d 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -163,15 +164,10 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) ascii_grid_symbols; /// Create separators - std::stringstream top_separator; - std::stringstream middle_names_separator; - std::stringstream middle_values_separator; - std::stringstream bottom_separator; - - top_separator.exceptions(std::ios::failbit); - middle_names_separator.exceptions(std::ios::failbit); - middle_values_separator.exceptions(std::ios::failbit); - bottom_separator.exceptions(std::ios::failbit); + WriteBufferFromOwnString top_separator; + WriteBufferFromOwnString middle_names_separator; + WriteBufferFromOwnString middle_values_separator; + WriteBufferFromOwnString bottom_separator; top_separator << grid_symbols.bold_left_top_corner; middle_names_separator << grid_symbols.bold_left_separator; diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 9320b159836..cfa669ae8ad 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -132,22 +133,17 @@ void PrettyCompactBlockOutputFormat::writeBottom(const Widths & max_widths) const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : ascii_grid_symbols; - /// Create delimiters - std::stringstream bottom_separator; - bottom_separator.exceptions(std::ios::failbit); - - bottom_separator << grid_symbols.left_bottom_corner; + /// Write delimiters + out << grid_symbols.left_bottom_corner; for (size_t i = 0; i < max_widths.size(); ++i) { if (i != 0) - bottom_separator << grid_symbols.bottom_separator; + out << grid_symbols.bottom_separator; for (size_t j = 0; j < max_widths[i] + 2; ++j) - bottom_separator << grid_symbols.dash; + out << grid_symbols.dash; } - bottom_separator << grid_symbols.right_bottom_corner << "\n"; - - writeString(bottom_separator.str(), out); + out << grid_symbols.right_bottom_corner << "\n"; } void PrettyCompactBlockOutputFormat::writeRow( diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp index e7a7200ac34..8f6888b9a7a 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -41,8 +42,7 @@ void CollapsingSortedAlgorithm::reportIncorrectData() if (!log) return; - std::stringstream s; - s.exceptions(std::ios::failbit); + WriteBufferFromOwnString s; auto & sort_columns = *last_row.sort_columns; for (size_t i = 0, size = sort_columns.size(); i < size; ++i) { diff --git a/src/Server/ReplicasStatusHandler.cpp b/src/Server/ReplicasStatusHandler.cpp index bf7a3b8ab52..fc79ad9d134 100644 --- a/src/Server/ReplicasStatusHandler.cpp +++ b/src/Server/ReplicasStatusHandler.cpp @@ -36,8 +36,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request const MergeTreeSettings & settings = context.getReplicatedMergeTreeSettings(); bool ok = true; - std::stringstream message; - message.exceptions(std::ios::failbit); + WriteBufferFromOwnString message; auto databases = DatabaseCatalog::instance().getDatabases(); @@ -83,7 +82,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request } if (verbose) - response.send() << message.rdbuf(); + response.send() << message.str(); else { const char * data = "Ok.\n"; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 4dceb0aa905..f9d21b568b6 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -436,13 +436,9 @@ bool TCPHandler::readDataNext(const size_t & poll_interval, const int & receive_ double elapsed = watch.elapsedSeconds(); if (elapsed > receive_timeout) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - ss << "Timeout exceeded while receiving data from client."; - ss << " Waited for " << static_cast(elapsed) << " seconds,"; - ss << " timeout is " << receive_timeout << " seconds."; - - throw Exception(ss.str(), ErrorCodes::SOCKET_TIMEOUT); + throw Exception(ErrorCodes::SOCKET_TIMEOUT, + "Timeout exceeded while receiving data from client. Waited for {} seconds, timeout is {} seconds.", + static_cast(elapsed), receive_timeout); } } diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index ef7c02163bb..093e5cea8ce 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -147,8 +147,7 @@ void DistributedBlockOutputStream::writeAsync(const Block & block) std::string DistributedBlockOutputStream::getCurrentStateDescription() { - std::stringstream buffer; - buffer.exceptions(std::ios::failbit); + WriteBufferFromOwnString buffer; const auto & addresses = cluster->getShardsAddresses(); buffer << "Insertion status:\n"; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 03eef37802f..effd2869be2 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -246,10 +246,7 @@ Names StorageKafka::parseTopics(String topic_list) String StorageKafka::getDefaultClientId(const StorageID & table_id_) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - ss << VERSION_NAME << "-" << getFQDNOrHostName() << "-" << table_id_.database_name << "-" << table_id_.table_name; - return ss.str(); + return fmt::format("{}-{}-{}-{}", VERSION_NAME, getFQDNOrHostName(), table_id_.database_name, table_id_.table_name); } @@ -400,10 +397,7 @@ ConsumerBufferPtr StorageKafka::createReadBuffer(const size_t consumer_number) conf.set("group.id", group); if (num_consumers > 1) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - ss << client_id << "-" << consumer_number; - conf.set("client.id", ss.str()); + conf.set("client.id", fmt::format("{}-{}", client_id, consumer_number)); } else { diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 265bc01be49..7c29b1582a6 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 58bdbcdcdcd..dd141a68248 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -442,12 +442,9 @@ void MinimalisticDataPartChecksums::checkEqualImpl(const MinimalisticDataPartChe { if (num_compressed_files != rhs.num_compressed_files || num_uncompressed_files != rhs.num_uncompressed_files) { - std::stringstream error_msg; - error_msg.exceptions(std::ios::failbit); - error_msg << "Different number of files: " << rhs.num_compressed_files << " compressed (expected " << num_compressed_files << ")" - << " and " << rhs.num_uncompressed_files << " uncompressed ones (expected " << num_uncompressed_files << ")"; - - throw Exception(error_msg.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH); + throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, + "Different number of files: {} compressed (expected {}) and {} uncompressed ones (expected {})", + rhs.num_compressed_files, num_compressed_files, rhs.num_uncompressed_files, num_uncompressed_files); } Strings errors; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f2b7882cbc9..5ebc0e4d423 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -228,14 +228,8 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue()) { - std::stringstream exception_message; - exception_message.exceptions(std::ios::failbit); - exception_message << "Primary key ("; - for (size_t i = 0, size = primary_key_columns.size(); i < size; ++i) - exception_message << (i == 0 ? "" : ", ") << primary_key_columns[i]; - exception_message << ") is not used and setting 'force_primary_key' is set."; - - throw Exception(exception_message.str(), ErrorCodes::INDEX_NOT_USED); + throw Exception(ErrorCodes::INDEX_NOT_USED, "Primary key ({}) is not used and setting 'force_primary_key' is set.", + boost::algorithm::join(primary_key_columns, ", ")); } std::optional minmax_idx_condition; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index aea740a3a26..9263dab638a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -941,8 +941,7 @@ size_t ReplicatedMergeTreeQueue::getConflictsCountForRange( if (out_description) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString ss; ss << "Can't execute command for range " << range.getPartName() << " (entry " << entry.znode_name << "). "; ss << "There are " << conflicts.size() << " currently executing entries blocking it: "; for (const auto & conflict : conflicts) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index f8128e6223a..f9e3c0558d3 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -186,15 +186,10 @@ AMQP::ExchangeType StorageRabbitMQ::defineExchangeType(String exchange_type_) String StorageRabbitMQ::getTableBasedName(String name, const StorageID & table_id) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); - if (name.empty()) - ss << table_id.database_name << "_" << table_id.table_name; + return fmt::format("{}_{}", table_id.database_name, table_id.table_name); else - ss << name << "_" << table_id.database_name << "_" << table_id.table_name; - - return ss.str(); + return fmt::format("{}_{}_{}", name, table_id.database_name, table_id.table_name); } diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index ee08dd5a824..f81bcbc42d0 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include namespace DB @@ -32,12 +32,9 @@ namespace { if (names_and_types_set.find(column) == names_and_types_set.end()) { - std::string message = "Not found column "; - message += column.name + " " + column.type->getName(); - message += " in dictionary " + backQuote(dictionary_name) + ". "; - message += "There are only columns "; - message += StorageDictionary::generateNamesAndTypesDescription(dictionary_names_and_types); - throw Exception(message, ErrorCodes::THERE_IS_NO_COLUMN); + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Not found column {} {} in dictionary {}. There are only columns {}", + column.name, column.type->getName(), backQuote(dictionary_name), + StorageDictionary::generateNamesAndTypesDescription(dictionary_names_and_types)); } } } @@ -81,8 +78,7 @@ NamesAndTypesList StorageDictionary::getNamesAndTypes(const DictionaryStructure String StorageDictionary::generateNamesAndTypesDescription(const NamesAndTypesList & list) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString ss; bool first = true; for (const auto & name_and_type : list) { diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 5cc435f91fa..a4500e2aa7b 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -427,8 +428,7 @@ namespace String listOfColumns(const NamesAndTypesList & available_columns) { - std::stringstream ss; - ss.exceptions(std::ios::failbit); + WriteBufferFromOwnString ss; for (auto it = available_columns.begin(); it != available_columns.end(); ++it) { if (it != available_columns.begin()) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index fbeb188d649..1b47bd6c32d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1013,17 +1013,13 @@ bool StorageMergeTree::optimize( { if (!merge(true, partition_id, true, deduplicate, &disable_reason)) { - std::stringstream message; - message.exceptions(std::ios::failbit); - message << "Cannot OPTIMIZE table"; - if (!disable_reason.empty()) - message << ": " << disable_reason; - else - message << " by some reason."; - LOG_INFO(log, message.str()); + constexpr const char * message = "Cannot OPTIMIZE table: {}"; + if (disable_reason.empty()) + disable_reason = "unknown reason"; + LOG_INFO(log, message, disable_reason); if (context.getSettingsRef().optimize_throw_if_noop) - throw Exception(message.str(), ErrorCodes::CANNOT_ASSIGN_OPTIMIZE); + throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason); return false; } } @@ -1036,17 +1032,13 @@ bool StorageMergeTree::optimize( if (!merge(true, partition_id, final, deduplicate, &disable_reason)) { - std::stringstream message; - message.exceptions(std::ios::failbit); - message << "Cannot OPTIMIZE table"; - if (!disable_reason.empty()) - message << ": " << disable_reason; - else - message << " by some reason."; - LOG_INFO(log, message.str()); + constexpr const char * message = "Cannot OPTIMIZE table: {}"; + if (disable_reason.empty()) + disable_reason = "unknown reason"; + LOG_INFO(log, message, disable_reason); if (context.getSettingsRef().optimize_throw_if_noop) - throw Exception(message.str(), ErrorCodes::CANNOT_ASSIGN_OPTIMIZE); + throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason); return false; } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 17de704ba40..c9e1b92bbbb 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -460,16 +460,12 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( if (!inactive_replicas.empty()) { - std::stringstream exception_message; - exception_message.exceptions(std::ios::failbit); - exception_message << "Mutation is not finished because"; + if (inactive_replicas.empty()) + throw Exception(ErrorCodes::UNFINISHED, "Mutation is not finished, it will be done asynchronously"); - if (!inactive_replicas.empty()) - exception_message << " some replicas are inactive right now: " << boost::algorithm::join(inactive_replicas, ", "); - - exception_message << ". Mutation will be done asynchronously"; - - throw Exception(exception_message.str(), ErrorCodes::UNFINISHED); + throw Exception(ErrorCodes::UNFINISHED, + "Mutation is not finished because some replicas are inactive right now: {}. Mutation will be done asynchronously", + boost::algorithm::join(inactive_replicas, ", ")); } } @@ -1018,13 +1014,6 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) for (const String & name : parts_to_fetch) parts_to_fetch_blocks += get_blocks_count_in_data_part(name); - std::stringstream sanity_report; - sanity_report.exceptions(std::ios::failbit); - sanity_report << "There are " - << unexpected_parts.size() << " unexpected parts with " << unexpected_parts_rows << " rows (" - << unexpected_parts_nonnew << " of them is not just-written with " << unexpected_parts_rows << " rows), " - << parts_to_fetch.size() << " missing parts (with " << parts_to_fetch_blocks << " blocks)."; - /** We can automatically synchronize data, * if the ratio of the total number of errors to the total number of parts (minimum - on the local filesystem or in ZK) * is no more than some threshold (for example 50%). @@ -1041,20 +1030,26 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) const auto storage_settings_ptr = getSettings(); bool insane = unexpected_parts_rows > total_rows_on_filesystem * storage_settings_ptr->replicated_max_ratio_of_wrong_parts; + constexpr const char * sanity_report_fmt = "The local set of parts of table {} doesn't look like the set of parts in ZooKeeper: " + "{} rows of {} total rows in filesystem are suspicious. " + "There are {} unexpected parts with {} rows ({} of them is not just-written with {} rows), " + "{} missing parts (with {} blocks)."; + if (insane && !skip_sanity_checks) { - std::stringstream why; - why.exceptions(std::ios::failbit); - why << "The local set of parts of table " << getStorageID().getNameForLogs() << " doesn't look like the set of parts " - << "in ZooKeeper: " - << formatReadableQuantity(unexpected_parts_rows) << " rows of " << formatReadableQuantity(total_rows_on_filesystem) - << " total rows in filesystem are suspicious."; - - throw Exception(why.str() + " " + sanity_report.str(), ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS); + throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS, sanity_report_fmt, getStorageID().getNameForLogs(), + formatReadableQuantity(unexpected_parts_rows), formatReadableQuantity(total_rows_on_filesystem), + unexpected_parts.size(), unexpected_parts_rows, unexpected_parts_nonnew, unexpected_parts_nonnew_rows, + parts_to_fetch.size(), parts_to_fetch_blocks); } if (unexpected_parts_nonnew_rows > 0) - LOG_WARNING(log, sanity_report.str()); + { + LOG_WARNING(log, sanity_report_fmt, getStorageID().getNameForLogs(), + formatReadableQuantity(unexpected_parts_rows), formatReadableQuantity(total_rows_on_filesystem), + unexpected_parts.size(), unexpected_parts_rows, unexpected_parts_nonnew, unexpected_parts_nonnew_rows, + parts_to_fetch.size(), parts_to_fetch_blocks); + } /// Add to the queue jobs to pick up the missing parts from other replicas and remove from ZK the information that we have them. std::vector> exists_futures; @@ -1343,15 +1338,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) { - // Log source part names just in case - { - std::stringstream source_parts_msg; - source_parts_msg.exceptions(std::ios::failbit); - for (auto i : ext::range(0, entry.source_parts.size())) - source_parts_msg << (i != 0 ? ", " : "") << entry.source_parts[i]; - - LOG_TRACE(log, "Executing log entry to merge parts {} to {}", source_parts_msg.str(), entry.new_part_name); - } + LOG_TRACE(log, "Executing log entry to merge parts {} to {}", boost::algorithm::join(entry.source_parts, ", "), entry.new_part_name); const auto storage_settings_ptr = getSettings(); @@ -3813,13 +3800,11 @@ bool StorageReplicatedMergeTree::optimize( if (!selected) { - std::stringstream message; - message.exceptions(std::ios::failbit); - message << "Cannot select parts for optimization"; - if (!disable_reason.empty()) - message << ": " << disable_reason; - LOG_INFO(log, message.str()); - return handle_noop(message.str()); + constexpr const char * message_fmt = "Cannot select parts for optimization: {}"; + if (disable_reason.empty()) + disable_reason = "unknown reason"; + LOG_INFO(log, message_fmt, disable_reason); + return handle_noop(fmt::format(message_fmt, disable_reason)); } ReplicatedMergeTreeLogEntryData merge_entry; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 66c688f3195..e5c16dad958 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -35,8 +35,6 @@ #include #include -#include - namespace DB { diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 81d3303e262..f35fb1c8a34 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -1,4 +1,3 @@ -#include #include #include #include diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 22a07a4d284..a031490b88b 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -241,13 +241,9 @@ TableFunctionRemote::TableFunctionRemote(const std::string & name_, bool secure_ : name{name_}, secure{secure_} { is_cluster_function = (name == "cluster" || name == "clusterAllReplicas"); - - std::stringstream ss; - ss.exceptions(std::ios::failbit); - ss << "Table function '" << name + "' requires from 2 to " << (is_cluster_function ? 3 : 5) << " parameters" - << ": , , " - << (is_cluster_function ? "" : ", [username, [password]]."); - help_message = ss.str(); + help_message = fmt::format("Table function '{}' requires from 2 to {} parameters: " + ", , {}", + name, is_cluster_function ? 3 : 5, is_cluster_function ? "" : ", [username, [password]]."); } diff --git a/utils/check-mysql-binlog/main.cpp b/utils/check-mysql-binlog/main.cpp index 4ec40ac41cc..0d831b84dce 100644 --- a/utils/check-mysql-binlog/main.cpp +++ b/utils/check-mysql-binlog/main.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include static DB::MySQLReplication::BinlogEventPtr parseSingleEventBody( @@ -126,18 +127,20 @@ static int checkBinLogFile(const std::string & bin_path, bool exist_checksum) } catch (...) { - std::cerr << "Unable to parse MySQL binlog event. Code: " << DB::getCurrentExceptionCode() << ", Exception message: " - << DB::getCurrentExceptionMessage(false) << std::endl << ", Previous event: " << std::endl; - last_event->dump(std::cerr); - std::cerr << std::endl << ", Event header: " << std::endl; - last_header->dump(std::cerr); - std::cerr << std::endl; + DB::WriteBufferFromOStream cerr(std::cerr); + cerr << "Unable to parse MySQL binlog event. Code: " << DB::getCurrentExceptionCode() << ", Exception message: " + << DB::getCurrentExceptionMessage(false) << '\n' << ", Previous event: " << '\n'; + last_event->dump(cerr); + cerr << '\n' << ", Event header: " << '\n'; + last_header->dump(cerr); + cerr << '\n'; return DB::getCurrentExceptionCode(); } - std::cout << "Check passed. " << std::endl << "No exception was thrown." << std::endl << "The last binlog event: " << std::endl; - last_event->dump(std::cout); - std::cout << std::endl; + DB::WriteBufferFromOStream cout(std::cout); + cout << "Check passed. " << '\n' << "No exception was thrown." << '\n' << "The last binlog event: " << '\n'; + last_event->dump(cout); + cout << '\n'; return 0; } diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 65ed4cec67a..56fd6843693 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -107,4 +107,4 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P ' $' | grep -P '.' && echo "^ Trailing whitespaces." # Forbid stringstream because it's easy to use them incorrectly and hard to debug possible issues -find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep 'std::ostringstream\|std::istringstream' | grep -v "STYLE_CHECK_ALLOW_STD_STRING_STREAM" && echo "Use WriteBufferFromString or ReadBufferFromString instead of std::ostringstream or std::istringstream" +find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep 'std::[io]\?stringstream' | grep -v "STYLE_CHECK_ALLOW_STD_STRING_STREAM" && echo "Use WriteBufferFromOwnString or ReadBufferFromString instead of std::stringstream" diff --git a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp index d16df83d12f..ad465913313 100644 --- a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp +++ b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp @@ -9,7 +9,7 @@ #include "00825_protobuf_format_syntax2.pb.h" -void writeInsertDataQueryForInputTest(std::stringstream & delimited_messages, const std::string & table_name, const std::string & format_schema, std::ostream & out) +void writeInsertDataQueryForInputTest(std::stringstream & delimited_messages, const std::string & table_name, const std::string & format_schema, std::ostream & out) // STYLE_CHECK_ALLOW_STD_STRING_STREAM { out << "echo -ne '"; std::string bytes = delimited_messages.str(); @@ -27,7 +27,7 @@ void writeInsertDataQueryForInputTest(std::stringstream & delimited_messages, co void writeInsertDataQueriesForInputTest(std::ostream & out) { - std::stringstream ss; + std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM { Person person; person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); diff --git a/utils/zookeeper-cli/zookeeper-cli.cpp b/utils/zookeeper-cli/zookeeper-cli.cpp index 17a8c9f0da8..47423dcd202 100644 --- a/utils/zookeeper-cli/zookeeper-cli.cpp +++ b/utils/zookeeper-cli/zookeeper-cli.cpp @@ -80,7 +80,7 @@ int main(int argc, char ** argv) try { - std::stringstream ss(line); + std::stringstream ss(line); // STYLE_CHECK_ALLOW_STD_STRING_STREAM std::string cmd; ss >> cmd; From ac8b658a4ae4b5c12757cbe2b3d6499250e54495 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 23:25:29 +0300 Subject: [PATCH 74/94] Try fix ci. --- src/AggregateFunctions/QuantileExact.h | 30 ++++++++++++++++++++++--- src/AggregateFunctions/QuantileTiming.h | 12 +++++++++- src/Columns/ColumnArray.cpp | 14 ++++++++++-- src/Columns/ColumnDecimal.cpp | 16 +++++++++++-- src/Columns/ColumnDecimal.h | 13 ++++++++++- src/Columns/ColumnFixedString.cpp | 20 ++++++++++++++++- src/Columns/ColumnLowCardinality.cpp | 9 +++++++- src/Columns/ColumnString.cpp | 14 ++++++++++-- src/Columns/ColumnTuple.cpp | 9 +++++++- src/Columns/ColumnVector.cpp | 20 ++++++++++++++++- 10 files changed, 142 insertions(+), 15 deletions(-) diff --git a/src/AggregateFunctions/QuantileExact.h b/src/AggregateFunctions/QuantileExact.h index de909c27565..f18ef75b1ba 100644 --- a/src/AggregateFunctions/QuantileExact.h +++ b/src/AggregateFunctions/QuantileExact.h @@ -8,7 +8,9 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) + #include +#endif namespace DB { @@ -88,7 +90,11 @@ struct QuantileExact : QuantileExactBase> { size_t n = level < 1 ? level * array.size() : (array.size() - 1); +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_select(array.begin(), array.begin() + n, array.end()); /// NOTE You can think of the radix-select algorithm. +#else + std::nth_element(array.begin(), array.begin() + n, array.end()); /// NOTE You can think of the radix-select algorithm. +#endif return array[n]; } @@ -108,8 +114,11 @@ struct QuantileExact : QuantileExactBase> size_t n = level < 1 ? level * array.size() : (array.size() - 1); +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n, array.end()); - +#else + std::nth_element(array.begin() + prev_n, array.begin() + n, array.end()); +#endif result[indices[i]] = array[n]; prev_n = n; } @@ -145,7 +154,11 @@ struct QuantileExactExclusive : public QuantileExact else if (n < 1) return static_cast(array[0]); +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_select(array.begin(), array.begin() + n - 1, array.end()); +#else + std::nth_element(array.begin(), array.begin() + n - 1, array.end()); +#endif 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]); @@ -174,7 +187,11 @@ struct QuantileExactExclusive : public QuantileExact result[indices[i]] = static_cast(array[0]); else { +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n - 1, array.end()); +#else + std::nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end()); +#endif 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,8 +226,11 @@ struct QuantileExactInclusive : public QuantileExact return static_cast(array[array.size() - 1]); else if (n < 1) return static_cast(array[0]); - +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_select(array.begin(), array.begin() + n - 1, array.end()); +#else + std::nth_element(array.begin(), array.begin() + n - 1, array.end()); +#endif 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]); @@ -237,7 +257,11 @@ struct QuantileExactInclusive : public QuantileExact result[indices[i]] = static_cast(array[0]); else { +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n - 1, array.end()); +#else + std::nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end()); +#endif 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 28bcde5c140..a1a8aabb5dc 100644 --- a/src/AggregateFunctions/QuantileTiming.h +++ b/src/AggregateFunctions/QuantileTiming.h @@ -7,7 +7,9 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) + #include +#endif namespace DB { @@ -180,7 +182,11 @@ namespace detail /// Sorting an array will not be considered a violation of constancy. auto & array = elems; +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_select(array.begin(), array.begin() + n, array.end()); +#else + std::nth_element(array.begin(), array.begin() + n, array.end()); +#endif quantile = array[n]; } @@ -201,7 +207,11 @@ namespace detail ? level * elems.size() : (elems.size() - 1); +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n, array.end()); +#else + std::nth_element(array.begin() + prev_n, array.begin() + n, array.end()); +#endif result[level_index] = array[n]; prev_n = n; diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 6dbe755f0ba..9bf0f929b8b 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -20,7 +20,10 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) + #include +#endif + namespace DB { @@ -783,7 +786,11 @@ 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) +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less); +#else + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); +#endif else std::sort(res.begin(), res.end(), less); } @@ -835,8 +842,11 @@ void ColumnArray::updatePermutationImpl(size_t limit, Permutation & res, EqualRa return; /// Since then we are working inside the interval. - +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); +#else + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); +#endif 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 4285259a4f4..249dcfcf644 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -8,7 +8,10 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) + #include +#endif + #include @@ -194,12 +197,21 @@ void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColum /// Since then we are working inside the interval. if (reverse) +#if !defined(ARCADIA_BUILD) 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, + [this](size_t a, size_t b) { return data[a] > data[b]; }); +#endif else +#if !defined(ARCADIA_BUILD) 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, + [this](size_t a, size_t b) { return data[a] > data[b]; }); +#endif auto new_first = first; for (auto j = first + 1; j < limit; ++j) { diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 128b595803f..45fead7edad 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -7,7 +7,10 @@ #include #include #include -#include +#if !defined(ARCADIA_BUILD) + #include +#endif + namespace DB @@ -254,9 +257,17 @@ protected: sort_end = res.begin() + limit; if (reverse) +#if !defined(ARCADIA_BUILD) 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]; }); +#endif else +#if !defined(ARCADIA_BUILD) 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]; }); +#endif } }; diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 41e46a7fa98..b2d48619af9 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -10,7 +10,9 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) + #include +#endif #include @@ -158,9 +160,17 @@ void ColumnFixedString::getPermutation(bool reverse, size_t limit, int /*nan_dir if (limit) { if (reverse) +#if !defined(ARCADIA_BUILD) 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)); +#endif else +#if !defined(ARCADIA_BUILD) 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)); +#endif } else { @@ -218,9 +228,17 @@ void ColumnFixedString::updatePermutation(bool reverse, size_t limit, int, Permu /// Since then we are working inside the interval. if (reverse) +#if !defined(ARCADIA_BUILD) 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)); +#endif else +#if !defined(ARCADIA_BUILD) 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)); +#endif 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 9b87a409aaa..3514859842c 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -8,7 +8,10 @@ #include #include -#include + +#if !defined(ARCADIA_BUILD) + #include +#endif namespace DB { @@ -394,7 +397,11 @@ void ColumnLowCardinality::updatePermutationImpl(size_t limit, Permutation & res /// Since then we are working inside the interval. +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); +#else + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); +#endif 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 541863486a6..6ad13ec252b 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -10,7 +10,10 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) + #include +#endif + namespace DB { @@ -314,7 +317,11 @@ 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) +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less); +#else + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); +#endif else std::sort(res.begin(), res.end(), less); } @@ -365,8 +372,11 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR return; /// Since then we are working inside the interval. - +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); +#else + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); +#endif 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 cbe5c7e11cd..e873f7d6530 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -9,7 +9,10 @@ #include #include #include -#include +#if !defined(ARCADIA_BUILD) + #include +#endif + namespace DB { @@ -352,7 +355,11 @@ void ColumnTuple::getPermutationImpl(size_t limit, Permutation & res, LessOperat if (limit) { +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less); +#else + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); +#endif } else { diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index e9af38d6984..99d92e9efa2 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -17,7 +17,9 @@ #include #include #include -#include +#if !defined(ARCADIA_BUILD) + #include +#endif #ifdef __SSE2__ #include @@ -156,9 +158,17 @@ void ColumnVector::getPermutation(bool reverse, size_t limit, int nan_directi res[i] = i; if (reverse) +#if !defined(ARCADIA_BUILD) 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(), greater(*this, nan_direction_hint)); +#endif else +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint)); +#else + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint)); +#endif } else { @@ -254,9 +264,17 @@ void ColumnVector::updatePermutation(bool reverse, size_t limit, int nan_dire /// Since then, we are working inside the interval. if (reverse) +#if !defined(ARCADIA_BUILD) 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, greater(*this, nan_direction_hint)); +#endif else +#if !defined(ARCADIA_BUILD) miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this, nan_direction_hint)); +#else + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this, nan_direction_hint)); +#endif size_t new_first = first; for (size_t j = first + 1; j < limit; ++j) From 0cca51f4aa4386c520f9a841ae796d0adb7d2236 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 11 Nov 2020 02:44:09 +0300 Subject: [PATCH 75/94] Check for wrong DWARF expressions for CFA --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index 27026ef4a9c..198458b35f1 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 27026ef4a9c6c8cc956d1d131c4d794e24096981 +Subproject commit 198458b35f100da32bd3e74c2a3ce8d236db299b From d948bbd15c8f0b5d2a73680f8e8ccef223c06256 Mon Sep 17 00:00:00 2001 From: OuO Date: Wed, 11 Nov 2020 15:46:29 +0800 Subject: [PATCH 76/94] Update tips.md --- docs/zh/operations/tips.md | 34 +++++++++++++++++----------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/docs/zh/operations/tips.md b/docs/zh/operations/tips.md index 4a8cf2efb96..511e8a22644 100644 --- a/docs/zh/operations/tips.md +++ b/docs/zh/operations/tips.md @@ -11,9 +11,9 @@ 不要禁用超线程。 它有助于某些查询,但不适用于其他查询。 -## 涡轮增压 {#turbo-boost} +## 超频 {#turbo-boost} -强烈推荐涡轮增压。 它显着提高了典型负载的性能。 +强烈推荐超频(turbo-boost)。 它显着提高了典型负载的性能。 您可以使用 `turbostat` 要查看负载下的CPU的实际时钟速率。 ## CPU缩放调控器 {#cpu-scaling-governor} @@ -39,18 +39,18 @@ echo 'performance' | sudo tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_gover 始终禁用交换文件。 不这样做的唯一原因是,如果您使用的ClickHouse在您的个人笔记本电脑。 -## 巨大的页面 {#huge-pages} +## 大页(Huge Pages) {#huge-pages} -始终禁用透明巨大的页面。 它会干扰内存分alloc,从而导致显着的性能下降。 +始终禁用透明大页(transparent huge pages)。 它会干扰内存分alloc,从而导致显着的性能下降。 ``` bash echo 'never' | sudo tee /sys/kernel/mm/transparent_hugepage/enabled ``` -使用 `perf top` 观看内核中用于内存管理的时间。 -永久巨大的页面也不需要被分配。 +使用 `perf top` 观察内核中用于内存管理的时间。 +永久大页(permanent huge pages)也不需要被分配。 -## 存储子系统 {#storage-subsystem} +## 存储系统 {#storage-subsystem} 如果您的预算允许您使用SSD,请使用SSD。 如果没有,请使用硬盘。 SATA硬盘7200转就行了。 @@ -100,27 +100,27 @@ XFS也是合适的,但它还没有经过ClickHouse的彻底测试。 如果可能的话,至少使用一个10GB的网络。 1Gb也可以工作,但对于使用数十tb的数据修补副本或处理具有大量中间数据的分布式查询,情况会更糟。 -## 动物园管理员 {#zookeeper} +## Zookeeper {#zookeeper} 您可能已经将ZooKeeper用于其他目的。 您可以使用相同的zookeeper安装,如果它还没有超载。 -It’s best to use a fresh version of ZooKeeper – 3.4.9 or later. The version in stable Linux distributions may be outdated. +最好使用新版本的 Zookeeper – 3.4.9 或之后的版本. 稳定 Liunx 发行版中的 Zookeeper 版本可能是落后的。 -You should never use manually written scripts to transfer data between different ZooKeeper clusters, because the result will be incorrect for sequential nodes. Never use the «zkcopy» utility for the same reason: https://github.com/ksprojects/zkcopy/issues/15 +你永远不该使用自己手写的脚本在不同的 Zookeeper 集群之间转移数据, 这可能会导致序列节点的数据不正确。出于同样的原因,永远不要使用 zkcopy 工具: https://github.com/ksprojects/zkcopy/issues/15 如果要将现有ZooKeeper集群分为两个,正确的方法是增加其副本的数量,然后将其重新配置为两个独立的集群。 -不要在与ClickHouse相同的服务器上运行ZooKeeper。 由于ZooKeeper对延迟非常敏感,ClickHouse可能会利用所有可用的系统资源。 +不要在与ClickHouse相同的服务器上运行ZooKeeper。 因为ZooKeeper对延迟非常敏感,而ClickHouse可能会占用所有可用的系统资源。 -使用默认设置,ZooKeeper是一个定时炸弹: +默认设置下,ZooKeeper 就像是一个定时炸弹: -> 使用默认配置时,ZooKeeper服务器不会从旧快照和日志中删除文件(请参阅autopurge),这是操作员的责任。 +当使用默认配置时,ZooKeeper服务不会从旧快照和日志中删除文件(请参阅autopurge),这是操作员的责任。 -必须拆除炸弹 +必须拆除炸弹。 -下面的ZooKeeper(3.5.1)配置在Yandex中使用。梅地卡生产环境截至2017年5月20日: +下面的ZooKeeper(3.5.1)配置在 Yandex.Metrica 的生产环境中使用截至2017年5月20日: -动物园cfg: +zoo.cfg: ``` bash # http://hadoop.apache.org/zookeeper/docs/current/zookeeperAdmin.html @@ -222,7 +222,7 @@ JAVA_OPTS="-Xms{{ '{{' }} cluster.get('xms','128M') {{ '}}' }} \ -XX:+CMSParallelRemarkEnabled" ``` -盐初始化: +Salt init: description "zookeeper-{{ '{{' }} cluster['name'] {{ '}}' }} centralized coordination service" From 0410e383d9e562a53ab2c8a3904f88efc7dc46a9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 12:46:36 +0300 Subject: [PATCH 77/94] Try fix check. --- src/AggregateFunctions/QuantileExact.h | 2 +- src/AggregateFunctions/QuantileTiming.h | 2 +- src/Columns/ColumnArray.cpp | 2 +- src/Columns/ColumnDecimal.cpp | 2 +- src/Columns/ColumnDecimal.h | 2 +- src/Columns/ColumnFixedString.cpp | 2 +- src/Columns/ColumnLowCardinality.cpp | 2 +- src/Columns/ColumnString.cpp | 2 +- src/Columns/ColumnTuple.cpp | 2 +- src/Columns/ColumnVector.cpp | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/AggregateFunctions/QuantileExact.h b/src/AggregateFunctions/QuantileExact.h index f18ef75b1ba..a2acde97f90 100644 --- a/src/AggregateFunctions/QuantileExact.h +++ b/src/AggregateFunctions/QuantileExact.h @@ -9,7 +9,7 @@ #include #if !defined(ARCADIA_BUILD) - #include + #include // Y_IGNORE #endif namespace DB diff --git a/src/AggregateFunctions/QuantileTiming.h b/src/AggregateFunctions/QuantileTiming.h index a1a8aabb5dc..1a696088dd4 100644 --- a/src/AggregateFunctions/QuantileTiming.h +++ b/src/AggregateFunctions/QuantileTiming.h @@ -8,7 +8,7 @@ #include #if !defined(ARCADIA_BUILD) - #include + #include // Y_IGNORE #endif namespace DB diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 9bf0f929b8b..82d954df334 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -21,7 +21,7 @@ #include #if !defined(ARCADIA_BUILD) - #include + #include // Y_IGNORE #endif diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 249dcfcf644..7c3af5fe095 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -9,7 +9,7 @@ #include #include #if !defined(ARCADIA_BUILD) - #include + #include // Y_IGNORE #endif diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 45fead7edad..cf9af32318e 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -8,7 +8,7 @@ #include #include #if !defined(ARCADIA_BUILD) - #include + #include // Y_IGNORE #endif diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index b2d48619af9..c1724b37fe2 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -11,7 +11,7 @@ #include #if !defined(ARCADIA_BUILD) - #include + #include // Y_IGNORE #endif #include diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 3514859842c..f61062160b1 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -10,7 +10,7 @@ #include #if !defined(ARCADIA_BUILD) - #include + #include // Y_IGNORE #endif namespace DB diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 6ad13ec252b..49180919abb 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -11,7 +11,7 @@ #include #include #if !defined(ARCADIA_BUILD) - #include + #include // Y_IGNORE #endif diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index e873f7d6530..9130f563735 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -10,7 +10,7 @@ #include #include #if !defined(ARCADIA_BUILD) - #include + #include // Y_IGNORE #endif diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 99d92e9efa2..c02963e4c5a 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -18,7 +18,7 @@ #include #include #if !defined(ARCADIA_BUILD) - #include + #include // Y_IGNORE #endif #ifdef __SSE2__ From 57c7ad1833595bdd2b0837f5a68863ebcbfece2a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 12:47:48 +0300 Subject: [PATCH 78/94] Fix style. --- src/Columns/ColumnDecimal.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index cf9af32318e..abb49531e89 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -12,7 +12,6 @@ #endif - namespace DB { From fae862f2c374d320577e1dd43ae0f9469b1667c7 Mon Sep 17 00:00:00 2001 From: heng zhao Date: Wed, 11 Nov 2020 17:51:27 +0800 Subject: [PATCH 79/94] Update index.md fix mis-spellings --- docs/zh/engines/table-engines/log-family/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/engines/table-engines/log-family/index.md b/docs/zh/engines/table-engines/log-family/index.md index 64378a73daa..c45a68b2aae 100644 --- a/docs/zh/engines/table-engines/log-family/index.md +++ b/docs/zh/engines/table-engines/log-family/index.md @@ -46,6 +46,6 @@ toc_priority: 29 `Log` 引擎为表中的每一列使用不同的文件。`StripeLog` 将所有的数据存储在一个文件中。因此 `StripeLog` 引擎在操作系统中使用更少的描述符,但是 `Log` 引擎提供更高的读性能。 -`TingLog` 引擎是该系列中最简单的引擎并且提供了最少的功能和最低的性能。`TingLog` 引擎不支持并行读取和并发数据访问,并将每一列存储在不同的文件中。它比其余两种支持并行读取的引擎的读取速度更慢,并且使用了和 `Log` 引擎同样多的描述符。你可以在简单的低负载的情景下使用它。 +`TinyLog` 引擎是该系列中最简单的引擎并且提供了最少的功能和最低的性能。`TinyLog` 引擎不支持并行读取和并发数据访问,并将每一列存储在不同的文件中。它比其余两种支持并行读取的引擎的读取速度更慢,并且使用了和 `Log` 引擎同样多的描述符。你可以在简单的低负载的情景下使用它。 [来源文章](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) From 66a45814858b8b915e9a67b0e9142f2829d8caf3 Mon Sep 17 00:00:00 2001 From: heng zhao Date: Wed, 11 Nov 2020 18:03:48 +0800 Subject: [PATCH 80/94] Update log.md engine name should not be translated. --- docs/zh/engines/table-engines/log-family/log.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/engines/table-engines/log-family/log.md b/docs/zh/engines/table-engines/log-family/log.md index 4a3855c4bfa..5d3ee38d745 100644 --- a/docs/zh/engines/table-engines/log-family/log.md +++ b/docs/zh/engines/table-engines/log-family/log.md @@ -1,5 +1,5 @@ -# 日志 {#log} +# Log {#log} -日志与 TinyLog 的不同之处在于,«标记» 的小文件与列文件存在一起。这些标记写在每个数据块上,并且包含偏移量,这些偏移量指示从哪里开始读取文件以便跳过指定的行数。这使得可以在多个线程中读取表数据。对于并发数据访问,可以同时执行读取操作,而写入操作则阻塞读取和其它写入。Log 引擎不支持索引。同样,如果写入表失败,则该表将被破坏,并且从该表读取将返回错误。Log 引擎适用于临时数据,write-once 表以及测试或演示目的。 +`Log` 与 `TinyLog` 的不同之处在于,«标记» 的小文件与列文件存在一起。这些标记写在每个数据块上,并且包含偏移量,这些偏移量指示从哪里开始读取文件以便跳过指定的行数。这使得可以在多个线程中读取表数据。对于并发数据访问,可以同时执行读取操作,而写入操作则阻塞读取和其它写入。`Log`引擎不支持索引。同样,如果写入表失败,则该表将被破坏,并且从该表读取将返回错误。`Log`引擎适用于临时数据,write-once 表以及测试或演示目的。 [原始文章](https://clickhouse.tech/docs/zh/operations/table_engines/log/) From 7da5e92754cda0e74c988472d45a4b579539ad0d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 11 Nov 2020 13:41:12 +0300 Subject: [PATCH 81/94] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c9e1b92bbbb..e27b9e8ec81 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -460,9 +460,6 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( if (!inactive_replicas.empty()) { - if (inactive_replicas.empty()) - throw Exception(ErrorCodes::UNFINISHED, "Mutation is not finished, it will be done asynchronously"); - throw Exception(ErrorCodes::UNFINISHED, "Mutation is not finished because some replicas are inactive right now: {}. Mutation will be done asynchronously", boost::algorithm::join(inactive_replicas, ", ")); From 072f481234dfdbf770f80529371811b4a052aeb2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 15:01:16 +0300 Subject: [PATCH 82/94] Try fix tests. --- tests/queries/0_stateless/arcadia_skip_list.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 6420eadfc09..25468835db4 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -165,4 +165,5 @@ 01548_query_log_query_execution_ms 01552_dict_fixedstring 01555_system_distribution_queue_mask -01557_max_parallel_replicas_no_sample.sql \ No newline at end of file +01557_max_parallel_replicas_no_sample.sql +01525_select_with_offset_fetch_clause From e311759c5d01052adf1a347fb1dea8786983063b Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 11 Nov 2020 15:15:16 +0300 Subject: [PATCH 83/94] Abort multipart upload if no data was written to WriteBufferFromS3. (#16840) --- src/IO/WriteBufferFromS3.cpp | 35 +++++++++++++++++------ tests/integration/test_storage_s3/test.py | 34 ++++++++++++++++++++++ 2 files changed, 60 insertions(+), 9 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 1ca8d8988d9..a32aa4acdc9 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -9,6 +9,7 @@ # include # include # include +# include # include # include # include @@ -167,6 +168,25 @@ void WriteBufferFromS3::complete() { if (is_multipart) { + if (part_tags.empty()) + { + LOG_DEBUG(log, "Multipart upload has no data. Aborting it. Bucket: {}, Key: {}, Upload_id: {}", bucket, key, upload_id); + + Aws::S3::Model::AbortMultipartUploadRequest req; + req.SetBucket(bucket); + req.SetKey(key); + req.SetUploadId(upload_id); + + auto outcome = client_ptr->AbortMultipartUpload(req); + + if (outcome.IsSuccess()) + LOG_DEBUG(log, "Aborting multipart upload completed. Upload_id: {}", upload_id); + else + throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + + return; + } + LOG_DEBUG(log, "Completing multipart upload. Bucket: {}, Key: {}, Upload_id: {}", bucket, key, upload_id); Aws::S3::Model::CompleteMultipartUploadRequest req; @@ -174,18 +194,15 @@ void WriteBufferFromS3::complete() req.SetKey(key); req.SetUploadId(upload_id); - if (!part_tags.empty()) + Aws::S3::Model::CompletedMultipartUpload multipart_upload; + for (size_t i = 0; i < part_tags.size(); ++i) { - Aws::S3::Model::CompletedMultipartUpload multipart_upload; - for (size_t i = 0; i < part_tags.size(); ++i) - { - Aws::S3::Model::CompletedPart part; - multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(i + 1)); - } - - req.SetMultipartUpload(multipart_upload); + Aws::S3::Model::CompletedPart part; + multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(i + 1)); } + req.SetMultipartUpload(multipart_upload); + auto outcome = client_ptr->CompleteMultipartUpload(req); if (outcome.IsSuccess()) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 42dbeda4717..4a2cd77e233 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -136,6 +136,40 @@ def test_put(cluster, maybe_auth, positive): assert values_csv == get_s3_file_content(cluster, bucket, filename) +# Test put no data to S3. +@pytest.mark.parametrize("auth", [ + "'minio','minio123'," +]) +def test_empty_put(cluster, auth): + # type: (ClickHouseCluster) -> None + + bucket = cluster.minio_bucket + instance = cluster.instances["dummy"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + create_empty_table_query = """ + CREATE TABLE empty_table ( + {} + ) ENGINE = Null() + """.format(table_format) + + run_query(instance, create_empty_table_query) + + filename = "empty_put_test.csv" + put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') select * from empty_table".format( + cluster.minio_host, cluster.minio_port, bucket, filename, auth, table_format) + + run_query(instance, put_query) + + try: + run_query(instance, "select count(*) from s3('http://{}:{}/{}/{}', {}'CSV', '{}')".format( + cluster.minio_host, cluster.minio_port, bucket, filename, auth, table_format)) + + assert False, "Query should be failed." + except helpers.client.QueryRuntimeException as e: + assert str(e).find("The specified key does not exist") != 0 + + # Test put values in CSV format. @pytest.mark.parametrize("maybe_auth,positive", [ ("", True), From 55d05c95bfcd43c9b094da5ff389d7be1f0ad365 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 11 Nov 2020 15:34:28 +0300 Subject: [PATCH 84/94] fixed style, xz check fasttest skipped, removed fast-lzma2 --- contrib/fast-lzma2 | 1 - docker/test/fasttest/run.sh | 1 + src/Server/HTTPHandler.cpp | 277 ++++++++++++++++++------------------ 3 files changed, 137 insertions(+), 142 deletions(-) delete mode 160000 contrib/fast-lzma2 diff --git a/contrib/fast-lzma2 b/contrib/fast-lzma2 deleted file mode 160000 index ded964d203c..00000000000 --- a/contrib/fast-lzma2 +++ /dev/null @@ -1 +0,0 @@ -Subproject commit ded964d203cabe1a572d2c813c55e8a94b4eda48 diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 36aa8baf252..5b2f324e588 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -268,6 +268,7 @@ TESTS_TO_SKIP=( protobuf secure sha256 + xz # Not sure why these two fail even in sequential mode. Disabled for now # to make some progress. diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index d7a1484d51d..94d66d44af0 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -5,39 +5,39 @@ #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include -#include #include #include #include -#include +#include #include -#include +#include +#include #include #include -#include -#include #include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #if !defined(ARCADIA_BUILD) # include @@ -46,8 +46,10 @@ namespace DB { + namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int CANNOT_PARSE_TEXT; extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; @@ -107,25 +109,36 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti { return HTTPResponse::HTTP_UNAUTHORIZED; } - else if ( - exception_code == ErrorCodes::CANNOT_PARSE_TEXT || exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE - || exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || exception_code == ErrorCodes::CANNOT_PARSE_DATE - || exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || exception_code == ErrorCodes::CANNOT_PARSE_NUMBER - || exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST - || exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || exception_code == ErrorCodes::TOO_DEEP_AST - || exception_code == ErrorCodes::TOO_BIG_AST || exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE - || exception_code == ErrorCodes::SYNTAX_ERROR || exception_code == ErrorCodes::INCORRECT_DATA - || exception_code == ErrorCodes::TYPE_MISMATCH) + else if (exception_code == ErrorCodes::CANNOT_PARSE_TEXT || + exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || + exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || + exception_code == ErrorCodes::CANNOT_PARSE_DATE || + exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || + exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || + exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || + exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || + exception_code == ErrorCodes::TOO_DEEP_AST || + exception_code == ErrorCodes::TOO_BIG_AST || + exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || + exception_code == ErrorCodes::SYNTAX_ERROR || + exception_code == ErrorCodes::INCORRECT_DATA || + exception_code == ErrorCodes::TYPE_MISMATCH) { return HTTPResponse::HTTP_BAD_REQUEST; } - else if ( - exception_code == ErrorCodes::UNKNOWN_TABLE || exception_code == ErrorCodes::UNKNOWN_FUNCTION - || exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || exception_code == ErrorCodes::UNKNOWN_TYPE - || exception_code == ErrorCodes::UNKNOWN_STORAGE || exception_code == ErrorCodes::UNKNOWN_DATABASE - || exception_code == ErrorCodes::UNKNOWN_SETTING || exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING - || exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || exception_code == ErrorCodes::UNKNOWN_FORMAT - || exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY) + else if (exception_code == ErrorCodes::UNKNOWN_TABLE || + exception_code == ErrorCodes::UNKNOWN_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || + exception_code == ErrorCodes::UNKNOWN_TYPE || + exception_code == ErrorCodes::UNKNOWN_STORAGE || + exception_code == ErrorCodes::UNKNOWN_DATABASE || + exception_code == ErrorCodes::UNKNOWN_SETTING || + exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || + exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_FORMAT || + exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY) { return HTTPResponse::HTTP_NOT_FOUND; } @@ -137,7 +150,8 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti { return HTTPResponse::HTTP_NOT_IMPLEMENTED; } - else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || exception_code == ErrorCodes::CANNOT_OPEN_FILE) + else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || + exception_code == ErrorCodes::CANNOT_OPEN_FILE) { return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; } @@ -150,7 +164,9 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti } -static std::chrono::steady_clock::duration parseSessionTimeout(const Poco::Util::AbstractConfiguration & config, const HTMLForm & params) +static std::chrono::steady_clock::duration parseSessionTimeout( + const Poco::Util::AbstractConfiguration & config, + const HTMLForm & params) { unsigned session_timeout = config.getInt("default_session_timeout", 60); @@ -164,9 +180,8 @@ static std::chrono::steady_clock::duration parseSessionTimeout(const Poco::Util: throw Exception("Invalid session timeout: '" + session_timeout_str + "'", ErrorCodes::INVALID_SESSION_TIMEOUT); if (session_timeout > max_session_timeout) - throw Exception( - "Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) - + ". Maximum session timeout could be modified in configuration file.", + throw Exception("Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) + + ". Maximum session timeout could be modified in configuration file.", ErrorCodes::INVALID_SESSION_TIMEOUT); } @@ -194,7 +209,8 @@ void HTTPHandler::pushDelayedResults(Output & used_output) IReadableWriteBuffer * write_buf_concrete; ReadBufferPtr reread_buf; - if (write_buf && (write_buf_concrete = dynamic_cast(write_buf.get())) + if (write_buf + && (write_buf_concrete = dynamic_cast(write_buf.get())) && (reread_buf = write_buf_concrete->tryGetReadBuffer())) { read_buffers.emplace_back(reread_buf); @@ -207,7 +223,9 @@ void HTTPHandler::pushDelayedResults(Output & used_output) } -HTTPHandler::HTTPHandler(IServer & server_, const std::string & name) : server(server_), log(&Poco::Logger::get(name)) +HTTPHandler::HTTPHandler(IServer & server_, const std::string & name) + : server(server_) + , log(&Poco::Logger::get(name)) { server_display_name = server.config().getString("display_name", getFQDNOrHostName()); } @@ -253,12 +271,12 @@ void HTTPHandler::processQuery( else { /// It is prohibited to mix different authorization schemes. - if (request.hasCredentials() || params.has("user") || params.has("password") || params.has("quota_key")) + if (request.hasCredentials() + || params.has("user") + || params.has("password") + || params.has("quota_key")) { - throw Exception( - "Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods " - "simultaneously", - ErrorCodes::REQUIRED_PASSWORD); + throw Exception("Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods simultaneously", ErrorCodes::REQUIRED_PASSWORD); } } @@ -300,13 +318,12 @@ void HTTPHandler::processQuery( { std::string opentelemetry_traceparent = request.get("traceparent"); std::string error; - if (!context.getClientInfo().parseTraceparentHeader(opentelemetry_traceparent, error)) + if (!context.getClientInfo().parseTraceparentHeader( + opentelemetry_traceparent, error)) { - throw Exception( - ErrorCodes::BAD_REQUEST_PARAMETER, + throw Exception(ErrorCodes::BAD_REQUEST_PARAMETER, "Failed to parse OpenTelemetry traceparent header '{}': {}", - opentelemetry_traceparent, - error); + opentelemetry_traceparent, error); } context.getClientInfo().opentelemetry_tracestate = request.get("tracestate", ""); @@ -315,7 +332,8 @@ void HTTPHandler::processQuery( // Set the query id supplied by the user, if any, and also update the // OpenTelemetry fields. - context.setCurrentQueryId(params.get("query_id", request.get("X-ClickHouse-Query-Id", ""))); + context.setCurrentQueryId(params.get("query_id", + request.get("X-ClickHouse-Query-Id", ""))); /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). String http_response_compression_methods = request.get("Accept-Encoding", ""); @@ -344,8 +362,8 @@ void HTTPHandler::processQuery( bool internal_compression = params.getParsed("compress", false); /// At least, we should postpone sending of first buffer_size result bytes - size_t buffer_size_total - = std::max(params.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); + size_t buffer_size_total = std::max( + params.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); /// If it is specified, the whole result will be buffered. /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. @@ -377,20 +395,23 @@ void HTTPHandler::processQuery( const std::string tmp_path(context.getTemporaryVolume()->getDisk()->getPath()); const std::string tmp_path_template(tmp_path + "http_buffers/"); - auto create_tmp_disk_buffer - = [tmp_path_template](const WriteBufferPtr &) { return WriteBufferFromTemporaryFile::create(tmp_path_template); }; + auto create_tmp_disk_buffer = [tmp_path_template] (const WriteBufferPtr &) + { + return WriteBufferFromTemporaryFile::create(tmp_path_template); + }; cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); } else { - auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed](const WriteBufferPtr & prev_buf) { + auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed] (const WriteBufferPtr & prev_buf) + { auto * prev_memory_buffer = typeid_cast(prev_buf.get()); if (!prev_memory_buffer) throw Exception("Expected MemoryWriteBuffer", ErrorCodes::LOGICAL_ERROR); auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); - copyData(*rdbuf, *next_buffer); + copyData(*rdbuf , *next_buffer); return next_buffer; }; @@ -398,8 +419,8 @@ void HTTPHandler::processQuery( cascade_buffer2.emplace_back(push_memory_buffer_and_continue); } - used_output.out_maybe_delayed_and_compressed - = std::make_shared(std::move(cascade_buffer1), std::move(cascade_buffer2)); + used_output.out_maybe_delayed_and_compressed = std::make_shared( + std::move(cascade_buffer1), std::move(cascade_buffer2)); } else { @@ -425,23 +446,13 @@ void HTTPHandler::processQuery( std::unique_ptr in; - static const NameSet reserved_param_names{ - "compress", - "decompress", - "user", - "password", - "quota_key", - "query_id", - "stacktrace", - "buffer_size", - "wait_end_of_query", - "session_id", - "session_timeout", - "session_check"}; + static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", + "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check"}; Names reserved_param_suffixes; - auto param_could_be_skipped = [&](const String & name) { + auto param_could_be_skipped = [&] (const String & name) + { /// Empty parameter appears when URL like ?&a=b or a=b&&c=d. Just skip them for user's convenience. if (name.empty()) return true; @@ -566,10 +577,12 @@ void HTTPHandler::processQuery( client_info.http_method = http_method; client_info.http_user_agent = request.get("User-Agent", ""); - auto append_callback = [&context](ProgressCallback callback) { + auto append_callback = [&context] (ProgressCallback callback) + { auto prev = context.getProgressCallback(); - context.setProgressCallback([prev, callback](const Progress & progress) { + context.setProgressCallback([prev, callback] (const Progress & progress) + { if (prev) prev(progress); @@ -579,13 +592,14 @@ void HTTPHandler::processQuery( /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. if (settings.send_progress_in_http_headers) - append_callback([&used_output](const Progress & progress) { used_output.out->onProgress(progress); }); + append_callback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); }); if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) { Poco::Net::StreamSocket & socket = dynamic_cast(request).socket(); - append_callback([&context, &socket](const Progress &) { + append_callback([&context, &socket](const Progress &) + { /// Assume that at the point this method is called no one is reading data from the socket any more. /// True for read-only queries. try @@ -609,17 +623,15 @@ void HTTPHandler::processQuery( query_scope.emplace(context); - executeQuery( - *in, - *used_output.out_maybe_delayed_and_compressed, - /* allow_into_outfile = */ false, - context, - [&response](const String & current_query_id, const String & content_type, const String & format, const String & timezone) { + executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & current_query_id, const String & content_type, const String & format, const String & timezone) + { response.setContentType(content_type); response.add("X-ClickHouse-Query-Id", current_query_id); response.add("X-ClickHouse-Format", format); response.add("X-ClickHouse-Timezone", timezone); - }); + } + ); if (used_output.hasDelayed()) { @@ -632,11 +644,8 @@ void HTTPHandler::processQuery( used_output.out->finalize(); } -void HTTPHandler::trySendExceptionToClient( - const std::string & s, - int exception_code, - Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response, +void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_code, + Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, Output & used_output) { try @@ -645,14 +654,17 @@ void HTTPHandler::trySendExceptionToClient( /// If HTTP method is POST and Keep-Alive is turned on, we should read the whole request body /// to avoid reading part of the current request body in the next request. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() && !request.stream().eof() + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST + && response.getKeepAlive() + && !request.stream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) { request.stream().ignore(std::numeric_limits::max()); } - bool auth_fail = exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD - || exception_code == ErrorCodes::REQUIRED_PASSWORD; + bool auth_fail = exception_code == ErrorCodes::UNKNOWN_USER || + exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::REQUIRED_PASSWORD; if (auth_fail) { @@ -730,12 +742,10 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne with_stacktrace = params.getParsed("stacktrace", false); /// Workaround. Poco does not detect 411 Length Required case. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() - && !request.hasContentLength()) + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && + !request.hasContentLength()) { - throw Exception( - "The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", - ErrorCodes::HTTP_LENGTH_REQUIRED); + throw Exception("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED); } processQuery(context, request, params, response, used_output, query_scope); @@ -763,7 +773,7 @@ DynamicQueryHandler::DynamicQueryHandler(IServer & server_, const std::string & bool DynamicQueryHandler::customizeQueryParam(Context & context, const std::string & key, const std::string & value) { if (key == param_name) - return true; /// do nothing + return true; /// do nothing if (startsWith(key, "param_")) { @@ -778,6 +788,7 @@ bool DynamicQueryHandler::customizeQueryParam(Context & context, const std::stri std::string DynamicQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) { + if (likely(!startsWith(request.getContentType(), "multipart/form-data"))) { /// Part of the query can be passed in the 'query' parameter and the rest in the request body @@ -802,16 +813,10 @@ std::string DynamicQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request } PredefinedQueryHandler::PredefinedQueryHandler( - IServer & server_, - const NameSet & receive_params_, - const std::string & predefined_query_, - const CompiledRegexPtr & url_regex_, - const std::unordered_map & header_name_with_regex_) - : HTTPHandler(server_, "PredefinedQueryHandler") - , receive_params(receive_params_) - , predefined_query(predefined_query_) - , url_regex(url_regex_) - , header_name_with_capture_regex(header_name_with_regex_) + IServer & server_, const NameSet & receive_params_, const std::string & predefined_query_ + , const CompiledRegexPtr & url_regex_, const std::unordered_map & header_name_with_regex_) + : HTTPHandler(server_, "PredefinedQueryHandler"), receive_params(receive_params_), predefined_query(predefined_query_) + , url_regex(url_regex_), header_name_with_capture_regex(header_name_with_regex_) { } @@ -831,7 +836,8 @@ void PredefinedQueryHandler::customizeContext(Poco::Net::HTTPServerRequest & req /// If in the configuration file, the handler's header is regex and contains named capture group /// We will extract regex named capture groups as query parameters - const auto & set_query_params = [&](const char * begin, const char * end, const CompiledRegexPtr & compiled_regex) { + const auto & set_query_params = [&](const char * begin, const char * end, const CompiledRegexPtr & compiled_regex) + { int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; re2::StringPiece matches[num_captures]; @@ -876,16 +882,16 @@ std::string PredefinedQueryHandler::getQuery(Poco::Net::HTTPServerRequest & requ Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & server, const std::string & config_prefix) { std::string query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query"); - return addFiltersFromConfig( - new HandlingRuleHTTPHandlerFactory(server, std::move(query_param_name)), server.config(), config_prefix); + return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory(server, std::move(query_param_name)), server.config(), config_prefix); } static inline bool capturingNamedQueryParam(NameSet receive_params, const CompiledRegexPtr & compiled_regex) { const auto & capturing_names = compiled_regex->NamedCapturingGroups(); - return std::count_if(capturing_names.begin(), capturing_names.end(), [&](const auto & iterator) { - return std::count_if( - receive_params.begin(), receive_params.end(), [&](const auto & param_name) { return param_name == iterator.first; }); + return std::count_if(capturing_names.begin(), capturing_names.end(), [&](const auto & iterator) + { + return std::count_if(receive_params.begin(), receive_params.end(), + [&](const auto & param_name) { return param_name == iterator.first; }); }); } @@ -894,10 +900,8 @@ static inline CompiledRegexPtr getCompiledRegex(const std::string & expression) auto compiled_regex = std::make_shared(expression); if (!compiled_regex->ok()) - throw Exception( - "Cannot compile re2: " + expression + " for http handling rule, error: " + compiled_regex->error() - + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", - ErrorCodes::CANNOT_COMPILE_REGEXP); + throw Exception("Cannot compile re2: " + expression + " for http handling rule, error: " + + compiled_regex->error() + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); return compiled_regex; } @@ -907,8 +911,7 @@ Poco::Net::HTTPRequestHandlerFactory * createPredefinedHandlerFactory(IServer & Poco::Util::AbstractConfiguration & configuration = server.config(); if (!configuration.has(config_prefix + ".handler.query")) - throw Exception( - "There is no path '" + config_prefix + ".handler.query" + "' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); + throw Exception("There is no path '" + config_prefix + ".handler.query" + "' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); std::string predefined_query = configuration.getString(config_prefix + ".handler.query"); NameSet analyze_receive_params = analyzeReceiveQueryParams(predefined_query); @@ -939,22 +942,14 @@ Poco::Net::HTTPRequestHandlerFactory * createPredefinedHandlerFactory(IServer & auto regex = getCompiledRegex(url_expression); if (capturingNamedQueryParam(analyze_receive_params, regex)) - return addFiltersFromConfig( - new HandlingRuleHTTPHandlerFactory( - server, - std::move(analyze_receive_params), - std::move(predefined_query), - std::move(regex), - std::move(headers_name_with_regex)), - configuration, - config_prefix); + return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory( + server, std::move(analyze_receive_params), std::move(predefined_query), std::move(regex), + std::move(headers_name_with_regex)), configuration, config_prefix); } - return addFiltersFromConfig( - new HandlingRuleHTTPHandlerFactory( - server, std::move(analyze_receive_params), std::move(predefined_query), CompiledRegexPtr{}, std::move(headers_name_with_regex)), - configuration, - config_prefix); + return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory( + server, std::move(analyze_receive_params), std::move(predefined_query), CompiledRegexPtr{} ,std::move(headers_name_with_regex)), + configuration, config_prefix); } } From 1b06fd9373c6cbaf16e082f0ffdd8d1e4700628f Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 11 Nov 2020 15:57:35 +0300 Subject: [PATCH 85/94] regenerated ya.make --- src/IO/ya.make | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ya.make b/src/IO/ya.make index 33b735fafa5..a4d406d73ce 100644 --- a/src/IO/ya.make +++ b/src/IO/ya.make @@ -20,9 +20,9 @@ SRCS( HTTPCommon.cpp HashingWriteBuffer.cpp HexWriteBuffer.cpp + LZMADeflatingWriteBuffer.cpp + LZMAInflatingReadBuffer.cpp LimitReadBuffer.cpp - LzmaReadBuffer.cpp - LzmaWriteBuffer.cpp MMapReadBufferFromFile.cpp MMapReadBufferFromFileDescriptor.cpp MemoryReadWriteBuffer.cpp From dbac7860b136d47342319cc28da28734d4c60d0e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 11 Nov 2020 16:27:54 +0300 Subject: [PATCH 86/94] review suggestions --- programs/client/QueryFuzzer.cpp | 1 + src/Common/MemoryTracker.cpp | 10 ++++++---- src/IO/WriteHelpers.h | 5 ----- 3 files changed, 7 insertions(+), 9 deletions(-) diff --git a/programs/client/QueryFuzzer.cpp b/programs/client/QueryFuzzer.cpp index cb058f6db00..a8e32d87db5 100644 --- a/programs/client/QueryFuzzer.cpp +++ b/programs/client/QueryFuzzer.cpp @@ -422,6 +422,7 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast) std::cout << std::endl; WriteBufferFromOStream ast_buf(std::cout, 4096); formatAST(*ast, ast_buf, false /*highlight*/); + ast_buf.next(); std::cout << std::endl << std::endl; } diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 1d324a00473..9f6db025886 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -136,8 +136,9 @@ void MemoryTracker::alloc(Int64 size) const auto * description = description_ptr.load(std::memory_order_relaxed); amount.fetch_sub(size, std::memory_order_relaxed); throw DB::Exception(DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory tracker {}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}", - description ? description : "", formatReadableSizeWithBinarySuffix(will_be), + "Memory tracker{}{}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}", + description ? " " : "", description ? description : "", + formatReadableSizeWithBinarySuffix(will_be), size, formatReadableSizeWithBinarySuffix(current_hard_limit)); } @@ -164,8 +165,9 @@ void MemoryTracker::alloc(Int64 size) const auto * description = description_ptr.load(std::memory_order_relaxed); amount.fetch_sub(size, std::memory_order_relaxed); throw DB::Exception(DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory limit {} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}", - description ? description : "", formatReadableSizeWithBinarySuffix(will_be), + "Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}", + description ? " " : "", description ? description : "", + formatReadableSizeWithBinarySuffix(will_be), size, formatReadableSizeWithBinarySuffix(current_hard_limit)); } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 4a117fefbee..dcf4e50e5af 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -271,11 +271,6 @@ inline void writeString(const StringRef & ref, WriteBuffer & buf) writeString(ref.data, ref.size, buf); } -//inline void writeString(const std::string_view & view, WriteBuffer & buf) -//{ -// writeString(view.data(), view.size(), buf); -//} - /** Writes a C-string without creating a temporary object. If the string is a literal, then `strlen` is executed at the compilation stage. * Use when the string is a literal. From bdbd41bd3db23e5b45a7a67dc646d4889f794e69 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Wed, 11 Nov 2020 18:29:36 +0300 Subject: [PATCH 87/94] Pytest test fixes (#16836) --- .../0_stateless/00039_inserts_through_http.sh | 2 + ...cated_merge_tree_alter_zookeeper.reference | 48 +- ..._replicated_merge_tree_alter_zookeeper.sql | 164 ++-- .../00070_insert_fewer_columns_http.sh | 1 + .../0_stateless/00386_long_in_pk.python | 2 + ...46_clear_column_in_partition_zookeeper.sql | 3 + ...ed_storage_definition_syntax_zookeeper.sql | 68 +- .../00643_cast_zookeeper.reference | 2 +- .../0_stateless/00643_cast_zookeeper.sql | 24 +- ..._create_temporary_table_with_in_clause.sql | 8 +- .../00719_insert_block_without_column.sh | 2 + ...fy_order_by_replicated_zookeeper.reference | 4 +- ...r_modify_order_by_replicated_zookeeper.sql | 46 +- ...dices_alter_replicated_zookeeper.reference | 24 +- ...836_indices_alter_replicated_zookeeper.sql | 114 +-- tests/queries/0_stateless/00900_orc_load.sh | 1 + ...om_compression_codecs_replicated.reference | 2 +- ...r_custom_compression_codecs_replicated.sql | 113 +-- .../00933_ttl_replicated_zookeeper.reference | 2 +- .../00933_ttl_replicated_zookeeper.sql | 31 +- ...nsert_into_distributed_different_types.sql | 3 + .../0_stateless/01056_create_table_as.sql | 4 + ..._materialize_clear_index_compact_parts.sql | 2 + ...1142_merge_join_lc_and_nullable_in_key.sql | 3 + ...01227_distributed_global_in_issue_2610.sql | 2 + ...ggregation_memory_efficient_mix_levels.sql | 5 + .../0_stateless/01231_operator_null_in.sql | 1 + tests/queries/0_stateless/01232_extremes.sql | 2 + ...ibuted_over_live_view_over_distributed.sql | 3 +- .../queries/0_stateless/01236_graphite_mt.sql | 10 +- .../0_stateless/01246_buffer_flush.sql | 2 + .../01246_finalize_aggregation_race.sql | 3 +- ...ted_group_by_sharding_key_dist_on_dist.sql | 1 + .../01268_data_numeric_parameters.sql | 4 + .../queries/0_stateless/01268_mv_scalars.sql | 2 + .../01272_totals_and_filter_bug.sql | 15 +- .../01273_lc_fixed_string_field.sql | 2 + .../queries/0_stateless/01275_parallel_mv.sql | 5 + .../0_stateless/01279_dist_group_by.sql | 2 + ..._max_threads_simple_query_optimization.sql | 2 + ...ence => 01283_strict_resize_bug.reference} | 0 ...ze_bug.sql => 01283_strict_resize_bug.sql} | 2 +- ...01285_data_skip_index_over_aggregation.sql | 2 + tests/queries/0_stateless/01291_geo_types.sql | 2 + .../01293_system_distribution_queue.sql | 3 + .../0_stateless/01296_pipeline_stuck.sql | 2 + tests/queries/0_stateless/01318_decrypt.sql | 2 + tests/queries/0_stateless/01318_encrypt.sql | 2 + .../0_stateless/01319_mv_constants_bug.sql | 1 + ...19_optimize_skip_unused_shards_nesting.sql | 4 + ...kip_unused_shards_no_non_deterministic.sql | 3 + ...num_partition_key_replicated_zookeeper.sql | 1 + .../0_stateless/01392_column_resolve.sql | 33 +- .../01400_join_get_with_multi_keys.sql | 1 + .../queries/0_stateless/01409_topK_merge.sql | 2 + .../01413_truncate_without_table_keyword.sql | 1 + .../01419_skip_index_compact_parts.sql | 4 +- .../01456_low_cardinality_sorting_bugfix.sql | 2 + ...457_create_as_table_function_structure.sql | 1 + .../01460_DistributedFilesToInsert.sql | 2 + .../01463_test_alter_live_view_refresh.sql | 3 + .../01487_distributed_in_not_default_db.sql | 1 + .../01511_prewhere_with_virtuals.sql | 4 +- ...3_optimize_aggregation_in_order_memory.sql | 2 + .../01514_empty_buffer_different_types.sql | 1 + .../01515_force_data_skipping_indices.sql | 2 + ...515_mv_and_array_join_optimisation_bag.sql | 10 +- .../01517_drop_mv_with_inner_table.sql | 7 + .../0_stateless/01526_initial_query_id.sh | 2 +- ...erministic_optimize_skip_unused_shards.sql | 2 + .../01530_drop_database_atomic_sync.sql | 8 +- .../01551_mergetree_read_in_order_spread.sql | 2 + tests/queries/conftest.py | 5 + tests/queries/query_test.py | 79 +- tests/queries/server.py | 842 ++++++++++++++++++ tests/queries/shell_config.sh | 3 + 76 files changed, 1403 insertions(+), 374 deletions(-) rename tests/queries/0_stateless/{001283_strict_resize_bug.reference => 01283_strict_resize_bug.reference} (100%) rename tests/queries/0_stateless/{001283_strict_resize_bug.sql => 01283_strict_resize_bug.sql} (90%) diff --git a/tests/queries/0_stateless/00039_inserts_through_http.sh b/tests/queries/0_stateless/00039_inserts_through_http.sh index 6b04437dcd6..35abcd166d7 100755 --- a/tests/queries/0_stateless/00039_inserts_through_http.sh +++ b/tests/queries/0_stateless/00039_inserts_through_http.sh @@ -10,3 +10,5 @@ for string_size in 1 10 100 1000 10000 100000 1000000; do LC_ALL=C perl -we 'for my $letter ("a" .. "z") { print(($letter x '$string_size') . "\n") }' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&query=INSERT+INTO+long_insert+FORMAT+TabSeparated" --data-binary @- echo 'SELECT substring(a, 1, 1) AS c, length(a) AS l FROM long_insert ORDER BY c, l' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @- done + +echo 'DROP TABLE long_insert' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @- diff --git a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference index 69887db71c1..cb61ab3e9b9 100644 --- a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference +++ b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference @@ -1,22 +1,22 @@ d Date k UInt64 i32 Int32 -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 10 42 d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\')\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 9 41 1992-01-01 08:00:00 2015-01-01 10 42 1970-01-01 00:00:00 d Date @@ -25,14 +25,14 @@ i32 Int32 dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] 2015-01-01 10 42 1970-01-01 00:00:00 [] [] @@ -43,7 +43,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -51,7 +51,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['1970-01-01','1970-01-01','1970-01-01'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] [] @@ -64,7 +64,7 @@ n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) s String DEFAULT \'0\' -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -73,7 +73,7 @@ n.ui8 Array(UInt8) n.s Array(String) n.d Array(Date) s String DEFAULT \'0\' -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `n.d` Array(Date),\n `s` String DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['1970-01-01','1970-01-01','1970-01-01'] 0 @@ -86,7 +86,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) s Int64 DEFAULT \'0\' -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -94,7 +94,7 @@ dt DateTime(\'UTC\') n.ui8 Array(UInt8) n.s Array(String) s Int64 DEFAULT \'0\' -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` Int64 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 @@ -108,7 +108,7 @@ n.ui8 Array(UInt8) n.s Array(String) s UInt32 DEFAULT \'0\' n.d Array(Date) -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -117,7 +117,7 @@ n.ui8 Array(UInt8) n.s Array(String) s UInt32 DEFAULT \'0\' n.d Array(Date) -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.ui8` Array(UInt8),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\',\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 ['1970-01-01','1970-01-01','1970-01-01'] 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 ['1970-01-01','1970-01-01','1970-01-01'] 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 ['1970-01-01','1970-01-01','1970-01-01'] @@ -129,14 +129,14 @@ i32 Int32 dt DateTime(\'UTC\') n.s Array(String) s UInt32 DEFAULT \'0\' -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') n.s Array(String) s UInt32 DEFAULT \'0\' -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `n.s` Array(String),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 ['asd','qwe','qwe'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 ['120','130','140'] 0 2015-01-01 8 40 2012-12-12 12:12:12 ['12','13','14'] 0 @@ -147,13 +147,13 @@ k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 2015-01-01 7 39 2014-07-14 13:26:50 0 2015-01-01 8 40 2012-12-12 12:12:12 0 @@ -166,7 +166,7 @@ dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' n.s Array(String) n.d Array(Date) -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 @@ -174,7 +174,7 @@ dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' n.s Array(String) n.d Array(Date) -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\',\n `n.s` Array(String),\n `n.d` Array(Date)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 [] [] 2015-01-01 7 39 2014-07-14 13:26:50 0 [] [] 2015-01-01 8 40 2012-12-12 12:12:12 0 [] [] @@ -185,13 +185,13 @@ k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt DateTime(\'UTC\') s UInt32 DEFAULT \'0\' -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` DateTime(\'UTC\'),\n `s` UInt32 DEFAULT \'0\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 2015-01-01 7 39 2014-07-14 13:26:50 0 2015-01-01 8 40 2012-12-12 12:12:12 0 @@ -202,13 +202,13 @@ k UInt64 i32 Int32 dt Date s DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\' -CREATE TABLE test.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) +CREATE TABLE default.replicated_alter1\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r1\', d, k, 8192) d Date k UInt64 i32 Int32 dt Date s DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\' -CREATE TABLE test.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) +CREATE TABLE default.replicated_alter2\n(\n `d` Date,\n `k` UInt64,\n `i32` Int32,\n `dt` Date,\n `s` DateTime(\'UTC\') DEFAULT \'1970-01-01 00:00:00\'\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00062/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 1970-01-02 03:55:00 2015-01-01 7 39 2014-07-14 1970-01-01 00:00:00 2015-01-01 8 40 2012-12-12 1970-01-01 00:00:00 diff --git a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql index 2a933d21eeb..ac56b3416cd 100644 --- a/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql +++ b/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.sql @@ -1,110 +1,110 @@ -DROP TABLE IF EXISTS test.replicated_alter1; -DROP TABLE IF EXISTS test.replicated_alter2; +DROP TABLE IF EXISTS replicated_alter1; +DROP TABLE IF EXISTS replicated_alter2; SET replication_alter_partitions_sync = 2; -CREATE TABLE test.replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00062/alter', 'r1', d, k, 8192); -CREATE TABLE test.replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00062/alter', 'r2', d, k, 8192); +CREATE TABLE replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00062/alter', 'r1', d, k, 8192); +CREATE TABLE replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_00062/alter', 'r2', d, k, 8192); -INSERT INTO test.replicated_alter1 VALUES ('2015-01-01', 10, 42); +INSERT INTO replicated_alter1 VALUES ('2015-01-01', 10, 42); -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -ALTER TABLE test.replicated_alter1 ADD COLUMN dt DateTime('UTC'); -INSERT INTO test.replicated_alter1 VALUES ('2015-01-01', 9, 41, '1992-01-01 08:00:00'); +ALTER TABLE replicated_alter1 ADD COLUMN dt DateTime('UTC'); +INSERT INTO replicated_alter1 VALUES ('2015-01-01', 9, 41, '1992-01-01 08:00:00'); -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -ALTER TABLE test.replicated_alter1 ADD COLUMN n Nested(ui8 UInt8, s String); -INSERT INTO test.replicated_alter1 VALUES ('2015-01-01', 8, 40, '2012-12-12 12:12:12', [1,2,3], ['12','13','14']); +ALTER TABLE replicated_alter1 ADD COLUMN n Nested(ui8 UInt8, s String); +INSERT INTO replicated_alter1 VALUES ('2015-01-01', 8, 40, '2012-12-12 12:12:12', [1,2,3], ['12','13','14']); -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -ALTER TABLE test.replicated_alter1 ADD COLUMN `n.d` Array(Date); -INSERT INTO test.replicated_alter1 VALUES ('2015-01-01', 7, 39, '2014-07-14 13:26:50', [10,20,30], ['120','130','140'],['2000-01-01','2000-01-01','2000-01-03']); +ALTER TABLE replicated_alter1 ADD COLUMN `n.d` Array(Date); +INSERT INTO replicated_alter1 VALUES ('2015-01-01', 7, 39, '2014-07-14 13:26:50', [10,20,30], ['120','130','140'],['2000-01-01','2000-01-01','2000-01-03']); -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -ALTER TABLE test.replicated_alter1 ADD COLUMN s String DEFAULT '0'; -INSERT INTO test.replicated_alter1 VALUES ('2015-01-01', 6,38,'2014-07-15 13:26:50',[10,20,30],['asd','qwe','qwe'],['2000-01-01','2000-01-01','2000-01-03'],'100500'); +ALTER TABLE replicated_alter1 ADD COLUMN s String DEFAULT '0'; +INSERT INTO replicated_alter1 VALUES ('2015-01-01', 6,38,'2014-07-15 13:26:50',[10,20,30],['asd','qwe','qwe'],['2000-01-01','2000-01-01','2000-01-03'],'100500'); -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -ALTER TABLE test.replicated_alter1 DROP COLUMN `n.d`, MODIFY COLUMN s Int64; +ALTER TABLE replicated_alter1 DROP COLUMN `n.d`, MODIFY COLUMN s Int64; -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -ALTER TABLE test.replicated_alter1 ADD COLUMN `n.d` Array(Date), MODIFY COLUMN s UInt32; +ALTER TABLE replicated_alter1 ADD COLUMN `n.d` Array(Date), MODIFY COLUMN s UInt32; -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -ALTER TABLE test.replicated_alter1 DROP COLUMN n.ui8, DROP COLUMN n.d; +ALTER TABLE replicated_alter1 DROP COLUMN n.ui8, DROP COLUMN n.d; -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -ALTER TABLE test.replicated_alter1 DROP COLUMN n.s; +ALTER TABLE replicated_alter1 DROP COLUMN n.s; -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -ALTER TABLE test.replicated_alter1 ADD COLUMN n.s Array(String), ADD COLUMN n.d Array(Date); +ALTER TABLE replicated_alter1 ADD COLUMN n.s Array(String), ADD COLUMN n.d Array(Date); -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -ALTER TABLE test.replicated_alter1 DROP COLUMN n; +ALTER TABLE replicated_alter1 DROP COLUMN n; -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -ALTER TABLE test.replicated_alter1 MODIFY COLUMN dt Date, MODIFY COLUMN s DateTime('UTC') DEFAULT '1970-01-01 00:00:00'; +ALTER TABLE replicated_alter1 MODIFY COLUMN dt Date, MODIFY COLUMN s DateTime('UTC') DEFAULT '1970-01-01 00:00:00'; -DESC TABLE test.replicated_alter1; -SHOW CREATE TABLE test.replicated_alter1; -DESC TABLE test.replicated_alter2; -SHOW CREATE TABLE test.replicated_alter2; -SELECT * FROM test.replicated_alter1 ORDER BY k; +DESC TABLE replicated_alter1; +SHOW CREATE TABLE replicated_alter1; +DESC TABLE replicated_alter2; +SHOW CREATE TABLE replicated_alter2; +SELECT * FROM replicated_alter1 ORDER BY k; -DROP TABLE test.replicated_alter1; -DROP TABLE test.replicated_alter2; +DROP TABLE replicated_alter1; +DROP TABLE replicated_alter2; diff --git a/tests/queries/0_stateless/00070_insert_fewer_columns_http.sh b/tests/queries/0_stateless/00070_insert_fewer_columns_http.sh index 853d4e161c9..0cf5f95d3d9 100755 --- a/tests/queries/0_stateless/00070_insert_fewer_columns_http.sh +++ b/tests/queries/0_stateless/00070_insert_fewer_columns_http.sh @@ -8,3 +8,4 @@ echo 'DROP TABLE IF EXISTS insert_fewer_columns' | ${ echo 'CREATE TABLE insert_fewer_columns (a UInt8, b UInt8) ENGINE = Memory' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @- echo 'INSERT INTO insert_fewer_columns (a) VALUES (1), (2)' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @- echo 'SELECT * FROM insert_fewer_columns' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @- +echo 'DROP TABLE insert_fewer_columns' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}" -d @- diff --git a/tests/queries/0_stateless/00386_long_in_pk.python b/tests/queries/0_stateless/00386_long_in_pk.python index ab5fc50d8e3..e33bb254c60 100644 --- a/tests/queries/0_stateless/00386_long_in_pk.python +++ b/tests/queries/0_stateless/00386_long_in_pk.python @@ -50,6 +50,8 @@ def main(): print(resp.text) break + requests.post(url, data='drop table tab_00386') + if __name__ == "__main__": main() diff --git a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql index bd6c12ffce4..5d8c4de1c06 100644 --- a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql @@ -71,3 +71,6 @@ OPTIMIZE TABLE clear_column1 PARTITION '200002'; ALTER TABLE clear_column1 CLEAR COLUMN s IN PARTITION '200012', CLEAR COLUMN i IN PARTITION '200012'; -- Drop empty partition also Ok ALTER TABLE clear_column1 DROP PARTITION '200012', DROP PARTITION '200011'; + +DROP TABLE clear_column1; +DROP TABLE clear_column2; diff --git a/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql b/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql index 4a3dd2981cd..4ed053f5953 100644 --- a/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql +++ b/tests/queries/0_stateless/00509_extended_storage_definition_syntax_zookeeper.sql @@ -1,81 +1,81 @@ SELECT '*** Replicated with sampling ***'; -DROP TABLE IF EXISTS test.replicated_with_sampling; +DROP TABLE IF EXISTS replicated_with_sampling; -CREATE TABLE test.replicated_with_sampling(x UInt8) +CREATE TABLE replicated_with_sampling(x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00509/replicated_with_sampling', 'r1') ORDER BY x SAMPLE BY x; -INSERT INTO test.replicated_with_sampling VALUES (1), (128); -SELECT sum(x) FROM test.replicated_with_sampling SAMPLE 1/2; +INSERT INTO replicated_with_sampling VALUES (1), (128); +SELECT sum(x) FROM replicated_with_sampling SAMPLE 1/2; -DROP TABLE test.replicated_with_sampling; +DROP TABLE replicated_with_sampling; SELECT '*** Replacing with implicit version ***'; -DROP TABLE IF EXISTS test.replacing; +DROP TABLE IF EXISTS replacing; -CREATE TABLE test.replacing(d Date, x UInt32, s String) ENGINE = ReplacingMergeTree ORDER BY x PARTITION BY d; +CREATE TABLE replacing(d Date, x UInt32, s String) ENGINE = ReplacingMergeTree ORDER BY x PARTITION BY d; -INSERT INTO test.replacing VALUES ('2017-10-23', 1, 'a'); -INSERT INTO test.replacing VALUES ('2017-10-23', 1, 'b'); -INSERT INTO test.replacing VALUES ('2017-10-23', 1, 'c'); +INSERT INTO replacing VALUES ('2017-10-23', 1, 'a'); +INSERT INTO replacing VALUES ('2017-10-23', 1, 'b'); +INSERT INTO replacing VALUES ('2017-10-23', 1, 'c'); -OPTIMIZE TABLE test.replacing PARTITION '2017-10-23' FINAL; +OPTIMIZE TABLE replacing PARTITION '2017-10-23' FINAL; -SELECT * FROM test.replacing; +SELECT * FROM replacing; -DROP TABLE test.replacing; +DROP TABLE replacing; SELECT '*** Replicated Collapsing ***'; -DROP TABLE IF EXISTS test.replicated_collapsing; +DROP TABLE IF EXISTS replicated_collapsing; -CREATE TABLE test.replicated_collapsing(d Date, x UInt32, sign Int8) +CREATE TABLE replicated_collapsing(d Date, x UInt32, sign Int8) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test_00509/replicated_collapsing', 'r1', sign) PARTITION BY toYYYYMM(d) ORDER BY d; -INSERT INTO test.replicated_collapsing VALUES ('2017-10-23', 1, 1); -INSERT INTO test.replicated_collapsing VALUES ('2017-10-23', 1, -1), ('2017-10-23', 2, 1); +INSERT INTO replicated_collapsing VALUES ('2017-10-23', 1, 1); +INSERT INTO replicated_collapsing VALUES ('2017-10-23', 1, -1), ('2017-10-23', 2, 1); -SYSTEM SYNC REPLICA test.replicated_collapsing; -OPTIMIZE TABLE test.replicated_collapsing PARTITION 201710 FINAL; +SYSTEM SYNC REPLICA replicated_collapsing; +OPTIMIZE TABLE replicated_collapsing PARTITION 201710 FINAL; -SELECT * FROM test.replicated_collapsing; +SELECT * FROM replicated_collapsing; -DROP TABLE test.replicated_collapsing; +DROP TABLE replicated_collapsing; SELECT '*** Replicated VersionedCollapsing ***'; -DROP TABLE IF EXISTS test.replicated_versioned_collapsing; +DROP TABLE IF EXISTS replicated_versioned_collapsing; -CREATE TABLE test.replicated_versioned_collapsing(d Date, x UInt32, sign Int8, version UInt8) +CREATE TABLE replicated_versioned_collapsing(d Date, x UInt32, sign Int8, version UInt8) ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/tables/test_00509/replicated_versioned_collapsing', 'r1', sign, version) PARTITION BY toYYYYMM(d) ORDER BY (d, version); -INSERT INTO test.replicated_versioned_collapsing VALUES ('2017-10-23', 1, 1, 0); -INSERT INTO test.replicated_versioned_collapsing VALUES ('2017-10-23', 1, -1, 0), ('2017-10-23', 2, 1, 0); -INSERT INTO test.replicated_versioned_collapsing VALUES ('2017-10-23', 1, -1, 1), ('2017-10-23', 2, 1, 2); +INSERT INTO replicated_versioned_collapsing VALUES ('2017-10-23', 1, 1, 0); +INSERT INTO replicated_versioned_collapsing VALUES ('2017-10-23', 1, -1, 0), ('2017-10-23', 2, 1, 0); +INSERT INTO replicated_versioned_collapsing VALUES ('2017-10-23', 1, -1, 1), ('2017-10-23', 2, 1, 2); -SYSTEM SYNC REPLICA test.replicated_versioned_collapsing; -OPTIMIZE TABLE test.replicated_versioned_collapsing PARTITION 201710 FINAL; +SYSTEM SYNC REPLICA replicated_versioned_collapsing; +OPTIMIZE TABLE replicated_versioned_collapsing PARTITION 201710 FINAL; -SELECT * FROM test.replicated_versioned_collapsing; +SELECT * FROM replicated_versioned_collapsing; -DROP TABLE test.replicated_versioned_collapsing; +DROP TABLE replicated_versioned_collapsing; SELECT '*** Table definition with SETTINGS ***'; -DROP TABLE IF EXISTS test.with_settings; +DROP TABLE IF EXISTS with_settings; -CREATE TABLE test.with_settings(x UInt32) +CREATE TABLE with_settings(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00509/with_settings', 'r1') ORDER BY x SETTINGS replicated_can_become_leader = 0; SELECT sleep(1); -- If replicated_can_become_leader were true, this replica would become the leader after 1 second. -SELECT is_leader FROM system.replicas WHERE database = 'test' AND table = 'with_settings'; +SELECT is_leader FROM system.replicas WHERE database = currentDatabase() AND table = 'with_settings'; -DROP TABLE test.with_settings; +DROP TABLE with_settings; diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.reference b/tests/queries/0_stateless/00643_cast_zookeeper.reference index eee4b660bda..9123463de1a 100644 --- a/tests/queries/0_stateless/00643_cast_zookeeper.reference +++ b/tests/queries/0_stateless/00643_cast_zookeeper.reference @@ -1,4 +1,4 @@ -CREATE TABLE test.cast1 +CREATE TABLE default.cast1 ( `x` UInt8, `e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)') diff --git a/tests/queries/0_stateless/00643_cast_zookeeper.sql b/tests/queries/0_stateless/00643_cast_zookeeper.sql index db439425eb8..c52d44bd88b 100644 --- a/tests/queries/0_stateless/00643_cast_zookeeper.sql +++ b/tests/queries/0_stateless/00643_cast_zookeeper.sql @@ -1,7 +1,7 @@ -DROP TABLE IF EXISTS test.cast1; -DROP TABLE IF EXISTS test.cast2; +DROP TABLE IF EXISTS cast1; +DROP TABLE IF EXISTS cast2; -CREATE TABLE test.cast1 +CREATE TABLE cast1 ( x UInt8, e Enum8 @@ -22,17 +22,17 @@ CREATE TABLE test.cast1 ) ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r1') ORDER BY e; -SHOW CREATE TABLE test.cast1 FORMAT TSVRaw; -DESC TABLE test.cast1; +SHOW CREATE TABLE cast1 FORMAT TSVRaw; +DESC TABLE cast1; -INSERT INTO test.cast1 (x) VALUES (1); -SELECT * FROM test.cast1; +INSERT INTO cast1 (x) VALUES (1); +SELECT * FROM cast1; -CREATE TABLE test.cast2 AS test.cast1 ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r2') ORDER BY e; +CREATE TABLE cast2 AS cast1 ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00643/cast', 'r2') ORDER BY e; -SYSTEM SYNC REPLICA test.cast2; +SYSTEM SYNC REPLICA cast2; -SELECT * FROM test.cast2; +SELECT * FROM cast2; -DROP TABLE test.cast1; -DROP TABLE test.cast2; +DROP TABLE cast1; +DROP TABLE cast2; diff --git a/tests/queries/0_stateless/00714_create_temporary_table_with_in_clause.sql b/tests/queries/0_stateless/00714_create_temporary_table_with_in_clause.sql index 751f766d0a1..e56c3bd1d7b 100644 --- a/tests/queries/0_stateless/00714_create_temporary_table_with_in_clause.sql +++ b/tests/queries/0_stateless/00714_create_temporary_table_with_in_clause.sql @@ -1,10 +1,10 @@ DROP TEMPORARY TABLE IF EXISTS temporary_table; -DROP TEMPORARY TABLE IF EXISTS test_merge_1; -DROP TEMPORARY TABLE IF EXISTS test_merge_2; +DROP TABLE IF EXISTS test_merge_1; +DROP TABLE IF EXISTS test_merge_2; CREATE TABLE test_merge_1(id UInt64) ENGINE = Log; CREATE TABLE test_merge_2(id UInt64) ENGINE = Log; CREATE TEMPORARY TABLE temporary_table AS SELECT * FROM numbers(1) WHERE number NOT IN (SELECT id FROM merge(currentDatabase(), 'test_merge_1|test_merge_2')); SELECT * FROM temporary_table; DROP TEMPORARY TABLE IF EXISTS temporary_table; -DROP TEMPORARY TABLE IF EXISTS test_merge_1; -DROP TEMPORARY TABLE IF EXISTS test_merge_2; +DROP TABLE IF EXISTS test_merge_1; +DROP TABLE IF EXISTS test_merge_2; diff --git a/tests/queries/0_stateless/00719_insert_block_without_column.sh b/tests/queries/0_stateless/00719_insert_block_without_column.sh index 5b7619e7b85..384445b1ae6 100755 --- a/tests/queries/0_stateless/00719_insert_block_without_column.sh +++ b/tests/queries/0_stateless/00719_insert_block_without_column.sh @@ -19,3 +19,5 @@ ${CLICKHOUSE_CLIENT} --query "create table squashed_numbers (SomeID UInt64, Diff ${CLICKHOUSE_CURL} -sS --data-binary "@${CLICKHOUSE_TMP}/test_squashing_block_without_column.out" "${CLICKHOUSE_URL}&query=insert%20into%20squashed_numbers%20format%20Native" ${CLICKHOUSE_CLIENT} --query "select 'Still alive'" + +${CLICKHOUSE_CLIENT} --query "drop table squashed_numbers" diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference index 4a7d4e3ff00..48fe2d30bf3 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference +++ b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference @@ -9,6 +9,6 @@ 1 2 1 30 1 2 4 90 *** Check SHOW CREATE TABLE *** -CREATE TABLE test.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, -z)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, -z)\nSETTINGS index_granularity = 8192 *** Check SHOW CREATE TABLE after offline ALTER *** -CREATE TABLE test.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `t` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, t * t)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.summing_r2\n(\n `x` UInt32,\n `y` UInt32,\n `z` UInt32,\n `t` UInt32,\n `val` UInt32\n)\nENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test_00754/summing\', \'r2\')\nPRIMARY KEY (x, y)\nORDER BY (x, y, t * t)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql index 0f861749537..4b150fb3826 100644 --- a/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql @@ -5,48 +5,48 @@ CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse ALTER TABLE old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36 } DROP TABLE old_style; -DROP TABLE IF EXISTS test.summing_r1; -DROP TABLE IF EXISTS test.summing_r2; -CREATE TABLE test.summing_r1(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test_00754/summing', 'r1') ORDER BY (x, y); -CREATE TABLE test.summing_r2(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test_00754/summing', 'r2') ORDER BY (x, y); +DROP TABLE IF EXISTS summing_r1; +DROP TABLE IF EXISTS summing_r2; +CREATE TABLE summing_r1(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test_00754/summing', 'r1') ORDER BY (x, y); +CREATE TABLE summing_r2(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test_00754/summing', 'r2') ORDER BY (x, y); /* Can't add an expression with existing column to ORDER BY. */ -ALTER TABLE test.summing_r1 MODIFY ORDER BY (x, y, -val); -- { serverError 36 } +ALTER TABLE summing_r1 MODIFY ORDER BY (x, y, -val); -- { serverError 36 } /* Can't add an expression with existing column to ORDER BY. */ -ALTER TABLE test.summing_r1 ADD COLUMN z UInt32 DEFAULT x + 1, MODIFY ORDER BY (x, y, -z); -- { serverError 36 } +ALTER TABLE summing_r1 ADD COLUMN z UInt32 DEFAULT x + 1, MODIFY ORDER BY (x, y, -z); -- { serverError 36 } /* Can't add nonexistent column to ORDER BY. */ -ALTER TABLE test.summing_r1 MODIFY ORDER BY (x, y, nonexistent); -- { serverError 47 } +ALTER TABLE summing_r1 MODIFY ORDER BY (x, y, nonexistent); -- { serverError 47 } /* Can't modyfy ORDER BY so that it is no longer a prefix of the PRIMARY KEY. */ -ALTER TABLE test.summing_r1 MODIFY ORDER BY x; -- { serverError 36 } +ALTER TABLE summing_r1 MODIFY ORDER BY x; -- { serverError 36 } -INSERT INTO test.summing_r1(x, y, val) VALUES (1, 2, 10), (1, 2, 20); -SYSTEM SYNC REPLICA test.summing_r2; +INSERT INTO summing_r1(x, y, val) VALUES (1, 2, 10), (1, 2, 20); +SYSTEM SYNC REPLICA summing_r2; -ALTER TABLE test.summing_r1 ADD COLUMN z UInt32 AFTER y, MODIFY ORDER BY (x, y, -z); +ALTER TABLE summing_r1 ADD COLUMN z UInt32 AFTER y, MODIFY ORDER BY (x, y, -z); -INSERT INTO test.summing_r1(x, y, z, val) values (1, 2, 1, 30), (1, 2, 2, 40), (1, 2, 2, 50); -SYSTEM SYNC REPLICA test.summing_r2; +INSERT INTO summing_r1(x, y, z, val) values (1, 2, 1, 30), (1, 2, 2, 40), (1, 2, 2, 50); +SYSTEM SYNC REPLICA summing_r2; SELECT '*** Check that the parts are sorted according to the new key. ***'; -SELECT * FROM test.summing_r2 ORDER BY _part; +SELECT * FROM summing_r2 ORDER BY _part; SELECT '*** Check that the rows are collapsed according to the new key. ***'; -SELECT * FROM test.summing_r2 FINAL ORDER BY x, y, z; +SELECT * FROM summing_r2 FINAL ORDER BY x, y, z; SELECT '*** Check SHOW CREATE TABLE ***'; -SHOW CREATE TABLE test.summing_r2; +SHOW CREATE TABLE summing_r2; -DETACH TABLE test.summing_r2; -ALTER TABLE test.summing_r1 ADD COLUMN t UInt32 AFTER z, MODIFY ORDER BY (x, y, t * t) SETTINGS replication_alter_partitions_sync = 2; -- { serverError 341 } -ATTACH TABLE test.summing_r2; +DETACH TABLE summing_r2; +ALTER TABLE summing_r1 ADD COLUMN t UInt32 AFTER z, MODIFY ORDER BY (x, y, t * t) SETTINGS replication_alter_partitions_sync = 2; -- { serverError 341 } +ATTACH TABLE summing_r2; -SYSTEM SYNC REPLICA test.summing_r2; +SYSTEM SYNC REPLICA summing_r2; SELECT '*** Check SHOW CREATE TABLE after offline ALTER ***'; -SHOW CREATE TABLE test.summing_r2; +SHOW CREATE TABLE summing_r2; -DROP TABLE test.summing_r1; -DROP TABLE test.summing_r2; +DROP TABLE summing_r1; +DROP TABLE summing_r2; diff --git a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference index 4029f0666b7..838bd93ebaf 100644 --- a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference +++ b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.reference @@ -1,5 +1,5 @@ -CREATE TABLE test.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE test.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 2 1 2 @@ -14,8 +14,8 @@ CREATE TABLE test.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDE 3 2 19 9 65 75 -CREATE TABLE test.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE test.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 4 1 5 @@ -28,10 +28,10 @@ CREATE TABLE test.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDE 3 2 19 9 65 75 -CREATE TABLE test.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE test.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE test.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE test.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter1\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 4 1 5 @@ -44,14 +44,14 @@ CREATE TABLE test.minmax_idx_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDE 3 2 19 9 65 75 -CREATE TABLE test.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE test.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32,\n INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10,\n INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 3 1 2 1 3 -CREATE TABLE test.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE test.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r1\')\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE default.minmax_idx2_r\n(\n `u64` UInt64,\n `i32` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00836/indices_alter2\', \'r2\')\nORDER BY u64\nSETTINGS index_granularity = 8192 1 2 1 3 1 2 diff --git a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql index 5773357c675..e038d2d425e 100644 --- a/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql @@ -1,74 +1,74 @@ -DROP TABLE IF EXISTS test.minmax_idx; -DROP TABLE IF EXISTS test.minmax_idx_r; -DROP TABLE IF EXISTS test.minmax_idx2; -DROP TABLE IF EXISTS test.minmax_idx2_r; +DROP TABLE IF EXISTS minmax_idx; +DROP TABLE IF EXISTS minmax_idx_r; +DROP TABLE IF EXISTS minmax_idx2; +DROP TABLE IF EXISTS minmax_idx2_r; SET replication_alter_partitions_sync = 2; -CREATE TABLE test.minmax_idx +CREATE TABLE minmax_idx ( u64 UInt64, i32 Int32 ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter1', 'r1') ORDER BY u64; -CREATE TABLE test.minmax_idx_r +CREATE TABLE minmax_idx_r ( u64 UInt64, i32 Int32 ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter1', 'r2') ORDER BY u64; -INSERT INTO test.minmax_idx VALUES (1, 2); +INSERT INTO minmax_idx VALUES (1, 2); -SYSTEM SYNC REPLICA test.minmax_idx_r; +SYSTEM SYNC REPLICA minmax_idx_r; -ALTER TABLE test.minmax_idx ADD INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10; -ALTER TABLE test.minmax_idx_r ADD INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10; -ALTER TABLE test.minmax_idx ADD INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10 AFTER idx1; +ALTER TABLE minmax_idx ADD INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10; +ALTER TABLE minmax_idx_r ADD INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10; +ALTER TABLE minmax_idx ADD INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10 AFTER idx1; -SHOW CREATE TABLE test.minmax_idx; -SHOW CREATE TABLE test.minmax_idx_r; +SHOW CREATE TABLE minmax_idx; +SHOW CREATE TABLE minmax_idx_r; -SELECT * FROM test.minmax_idx WHERE u64 * i32 = 2 ORDER BY (u64, i32); -SELECT * FROM test.minmax_idx_r WHERE u64 * i32 = 2 ORDER BY (u64, i32); +SELECT * FROM minmax_idx WHERE u64 * i32 = 2 ORDER BY (u64, i32); +SELECT * FROM minmax_idx_r WHERE u64 * i32 = 2 ORDER BY (u64, i32); -INSERT INTO test.minmax_idx VALUES (1, 4); -INSERT INTO test.minmax_idx_r VALUES (3, 2); -INSERT INTO test.minmax_idx VALUES (1, 5); -INSERT INTO test.minmax_idx_r VALUES (65, 75); -INSERT INTO test.minmax_idx VALUES (19, 9); +INSERT INTO minmax_idx VALUES (1, 4); +INSERT INTO minmax_idx_r VALUES (3, 2); +INSERT INTO minmax_idx VALUES (1, 5); +INSERT INTO minmax_idx_r VALUES (65, 75); +INSERT INTO minmax_idx VALUES (19, 9); -SYSTEM SYNC REPLICA test.minmax_idx; -SYSTEM SYNC REPLICA test.minmax_idx_r; +SYSTEM SYNC REPLICA minmax_idx; +SYSTEM SYNC REPLICA minmax_idx_r; -SELECT * FROM test.minmax_idx WHERE u64 * i32 > 1 ORDER BY (u64, i32); -SELECT * FROM test.minmax_idx_r WHERE u64 * i32 > 1 ORDER BY (u64, i32); +SELECT * FROM minmax_idx WHERE u64 * i32 > 1 ORDER BY (u64, i32); +SELECT * FROM minmax_idx_r WHERE u64 * i32 > 1 ORDER BY (u64, i32); -ALTER TABLE test.minmax_idx DROP INDEX idx1; +ALTER TABLE minmax_idx DROP INDEX idx1; -SHOW CREATE TABLE test.minmax_idx; -SHOW CREATE TABLE test.minmax_idx_r; +SHOW CREATE TABLE minmax_idx; +SHOW CREATE TABLE minmax_idx_r; -SELECT * FROM test.minmax_idx WHERE u64 * i32 > 1 ORDER BY (u64, i32); -SELECT * FROM test.minmax_idx_r WHERE u64 * i32 > 1 ORDER BY (u64, i32); +SELECT * FROM minmax_idx WHERE u64 * i32 > 1 ORDER BY (u64, i32); +SELECT * FROM minmax_idx_r WHERE u64 * i32 > 1 ORDER BY (u64, i32); -ALTER TABLE test.minmax_idx DROP INDEX idx2; -ALTER TABLE test.minmax_idx_r DROP INDEX idx3; +ALTER TABLE minmax_idx DROP INDEX idx2; +ALTER TABLE minmax_idx_r DROP INDEX idx3; -SHOW CREATE TABLE test.minmax_idx; -SHOW CREATE TABLE test.minmax_idx_r; +SHOW CREATE TABLE minmax_idx; +SHOW CREATE TABLE minmax_idx_r; -ALTER TABLE test.minmax_idx ADD INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10; +ALTER TABLE minmax_idx ADD INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10; -SHOW CREATE TABLE test.minmax_idx; -SHOW CREATE TABLE test.minmax_idx_r; +SHOW CREATE TABLE minmax_idx; +SHOW CREATE TABLE minmax_idx_r; -SELECT * FROM test.minmax_idx WHERE u64 * i32 > 1 ORDER BY (u64, i32); -SELECT * FROM test.minmax_idx_r WHERE u64 * i32 > 1 ORDER BY (u64, i32); +SELECT * FROM minmax_idx WHERE u64 * i32 > 1 ORDER BY (u64, i32); +SELECT * FROM minmax_idx_r WHERE u64 * i32 > 1 ORDER BY (u64, i32); -CREATE TABLE test.minmax_idx2 +CREATE TABLE minmax_idx2 ( u64 UInt64, i32 Int32, @@ -77,7 +77,7 @@ CREATE TABLE test.minmax_idx2 ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00836/indices_alter2', 'r1') ORDER BY u64; -CREATE TABLE test.minmax_idx2_r +CREATE TABLE minmax_idx2_r ( u64 UInt64, i32 Int32, @@ -87,27 +87,27 @@ CREATE TABLE test.minmax_idx2_r ORDER BY u64; -SHOW CREATE TABLE test.minmax_idx2; -SHOW CREATE TABLE test.minmax_idx2_r; +SHOW CREATE TABLE minmax_idx2; +SHOW CREATE TABLE minmax_idx2_r; -INSERT INTO test.minmax_idx2 VALUES (1, 2); -INSERT INTO test.minmax_idx2_r VALUES (1, 3); +INSERT INTO minmax_idx2 VALUES (1, 2); +INSERT INTO minmax_idx2_r VALUES (1, 3); -SYSTEM SYNC REPLICA test.minmax_idx2; -SYSTEM SYNC REPLICA test.minmax_idx2_r; +SYSTEM SYNC REPLICA minmax_idx2; +SYSTEM SYNC REPLICA minmax_idx2_r; -SELECT * FROM test.minmax_idx2 WHERE u64 * i32 >= 2 ORDER BY (u64, i32); -SELECT * FROM test.minmax_idx2_r WHERE u64 * i32 >= 2 ORDER BY (u64, i32); +SELECT * FROM minmax_idx2 WHERE u64 * i32 >= 2 ORDER BY (u64, i32); +SELECT * FROM minmax_idx2_r WHERE u64 * i32 >= 2 ORDER BY (u64, i32); -ALTER TABLE test.minmax_idx2_r DROP INDEX idx1, DROP INDEX idx2; +ALTER TABLE minmax_idx2_r DROP INDEX idx1, DROP INDEX idx2; -SHOW CREATE TABLE test.minmax_idx2; -SHOW CREATE TABLE test.minmax_idx2_r; +SHOW CREATE TABLE minmax_idx2; +SHOW CREATE TABLE minmax_idx2_r; -SELECT * FROM test.minmax_idx2 WHERE u64 * i32 >= 2 ORDER BY (u64, i32); -SELECT * FROM test.minmax_idx2_r WHERE u64 * i32 >= 2 ORDER BY (u64, i32); +SELECT * FROM minmax_idx2 WHERE u64 * i32 >= 2 ORDER BY (u64, i32); +SELECT * FROM minmax_idx2_r WHERE u64 * i32 >= 2 ORDER BY (u64, i32); -DROP TABLE test.minmax_idx; -DROP TABLE test.minmax_idx_r; -DROP TABLE test.minmax_idx2; -DROP TABLE test.minmax_idx2_r; +DROP TABLE minmax_idx; +DROP TABLE minmax_idx_r; +DROP TABLE minmax_idx2; +DROP TABLE minmax_idx2_r; diff --git a/tests/queries/0_stateless/00900_orc_load.sh b/tests/queries/0_stateless/00900_orc_load.sh index 5d2e45e2574..6e08b415397 100755 --- a/tests/queries/0_stateless/00900_orc_load.sh +++ b/tests/queries/0_stateless/00900_orc_load.sh @@ -10,3 +10,4 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (int Int32, smallint Int8, b cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" ${CLICKHOUSE_CLIENT} --query="select * from orc_load" +${CLICKHOUSE_CLIENT} --query="drop table orc_load" diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference index 328022414b8..29e7b23d3dd 100644 --- a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference +++ b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.reference @@ -20,7 +20,7 @@ 274972506.6 9175437371954010821 9175437371954010821 -CREATE TABLE test.compression_codec_multiple_more_types_replicated\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00910/compression_codec_multiple_more_types_replicated\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.compression_codec_multiple_more_types_replicated\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00910/compression_codec_multiple_more_types_replicated\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1.5555555555555 hello world! [77] ['John'] 7.1000000000000 xxxxxxxxxxxx [127] ['Henry'] ! diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql index e4c9f0ffc3b..52eb1d4e411 100644 --- a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql +++ b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated.sql @@ -1,10 +1,10 @@ SET send_logs_level = 'fatal'; SET allow_suspicious_codecs = 1; -DROP TABLE IF EXISTS test.compression_codec_replicated1; -DROP TABLE IF EXISTS test.compression_codec_replicated2; +DROP TABLE IF EXISTS compression_codec_replicated1; +DROP TABLE IF EXISTS compression_codec_replicated2; -CREATE TABLE test.compression_codec_replicated1( +CREATE TABLE compression_codec_replicated1( id UInt64 CODEC(LZ4), data String CODEC(ZSTD), ddd Date CODEC(NONE), @@ -13,7 +13,7 @@ CREATE TABLE test.compression_codec_replicated1( othernum Int64 CODEC(Delta) ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_replicated', '1') ORDER BY tuple(); -CREATE TABLE test.compression_codec_replicated2( +CREATE TABLE compression_codec_replicated2( id UInt64 CODEC(LZ4), data String CODEC(ZSTD), ddd Date CODEC(NONE), @@ -23,43 +23,43 @@ CREATE TABLE test.compression_codec_replicated2( ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_replicated', '2') ORDER BY tuple(); -INSERT INTO test.compression_codec_replicated1 VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); -INSERT INTO test.compression_codec_replicated1 VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6); -INSERT INTO test.compression_codec_replicated1 VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7); +INSERT INTO compression_codec_replicated1 VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); +INSERT INTO compression_codec_replicated1 VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6); +INSERT INTO compression_codec_replicated1 VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7); -SYSTEM SYNC REPLICA test.compression_codec_replicated2; +SYSTEM SYNC REPLICA compression_codec_replicated2; -SELECT * FROM test.compression_codec_replicated1 ORDER BY id; -SELECT * FROM test.compression_codec_replicated2 ORDER BY id; +SELECT * FROM compression_codec_replicated1 ORDER BY id; +SELECT * FROM compression_codec_replicated2 ORDER BY id; -OPTIMIZE TABLE test.compression_codec_replicated1 FINAL; +OPTIMIZE TABLE compression_codec_replicated1 FINAL; -INSERT INTO test.compression_codec_replicated1 VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8); +INSERT INTO compression_codec_replicated1 VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8); -SYSTEM SYNC REPLICA test.compression_codec_replicated2; +SYSTEM SYNC REPLICA compression_codec_replicated2; -DETACH TABLE test.compression_codec_replicated1; -ATTACH TABLE test.compression_codec_replicated1; +DETACH TABLE compression_codec_replicated1; +ATTACH TABLE compression_codec_replicated1; -SELECT count(*) FROM test.compression_codec_replicated1 WHERE id = 2 GROUP BY id; -SELECT count(*) FROM test.compression_codec_replicated2 WHERE id = 2 GROUP BY id; +SELECT count(*) FROM compression_codec_replicated1 WHERE id = 2 GROUP BY id; +SELECT count(*) FROM compression_codec_replicated2 WHERE id = 2 GROUP BY id; -DROP TABLE IF EXISTS test.compression_codec_replicated1; -DROP TABLE IF EXISTS test.compression_codec_replicated2; +DROP TABLE IF EXISTS compression_codec_replicated1; +DROP TABLE IF EXISTS compression_codec_replicated2; -DROP TABLE IF EXISTS test.compression_codec_multiple_replicated1; -DROP TABLE IF EXISTS test.compression_codec_multiple_replicated2; +DROP TABLE IF EXISTS compression_codec_multiple_replicated1; +DROP TABLE IF EXISTS compression_codec_multiple_replicated2; SET network_compression_method = 'lz4hc'; -CREATE TABLE test.compression_codec_multiple_replicated1 ( +CREATE TABLE compression_codec_multiple_replicated1 ( id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple', '1') ORDER BY tuple(); -CREATE TABLE test.compression_codec_multiple_replicated2 ( +CREATE TABLE compression_codec_multiple_replicated2 ( id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), @@ -67,72 +67,73 @@ CREATE TABLE test.compression_codec_multiple_replicated2 ( ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple', '2') ORDER BY tuple(); -INSERT INTO test.compression_codec_multiple_replicated2 VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); +INSERT INTO compression_codec_multiple_replicated2 VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); -SYSTEM SYNC REPLICA test.compression_codec_multiple_replicated1; +SYSTEM SYNC REPLICA compression_codec_multiple_replicated1; -SELECT * FROM test.compression_codec_multiple_replicated2 ORDER BY id; -SELECT * FROM test.compression_codec_multiple_replicated1 ORDER BY id; +SELECT * FROM compression_codec_multiple_replicated2 ORDER BY id; +SELECT * FROM compression_codec_multiple_replicated1 ORDER BY id; -INSERT INTO test.compression_codec_multiple_replicated1 select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number FROM system.numbers limit 10000; +INSERT INTO compression_codec_multiple_replicated1 select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number FROM system.numbers limit 10000; -SYSTEM SYNC REPLICA test.compression_codec_multiple_replicated2; +SYSTEM SYNC REPLICA compression_codec_multiple_replicated2; -SELECT count(*) FROM test.compression_codec_multiple_replicated1; -SELECT count(*) FROM test.compression_codec_multiple_replicated2; +SELECT count(*) FROM compression_codec_multiple_replicated1; +SELECT count(*) FROM compression_codec_multiple_replicated2; -SELECT count(distinct data) FROM test.compression_codec_multiple_replicated1; -SELECT count(distinct data) FROM test.compression_codec_multiple_replicated2; +SELECT count(distinct data) FROM compression_codec_multiple_replicated1; +SELECT count(distinct data) FROM compression_codec_multiple_replicated2; -SELECT floor(sum(somenum), 1) FROM test.compression_codec_multiple_replicated1; -SELECT floor(sum(somenum), 1) FROM test.compression_codec_multiple_replicated2; +SELECT floor(sum(somenum), 1) FROM compression_codec_multiple_replicated1; +SELECT floor(sum(somenum), 1) FROM compression_codec_multiple_replicated2; -TRUNCATE TABLE test.compression_codec_multiple_replicated1; -SYSTEM SYNC REPLICA test.compression_codec_multiple_replicated2; +TRUNCATE TABLE compression_codec_multiple_replicated1; +SYSTEM SYNC REPLICA compression_codec_multiple_replicated2; -INSERT INTO test.compression_codec_multiple_replicated1 select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number FROM system.numbers limit 10000; +INSERT INTO compression_codec_multiple_replicated1 select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number FROM system.numbers limit 10000; -SYSTEM SYNC REPLICA test.compression_codec_multiple_replicated2; +SYSTEM SYNC REPLICA compression_codec_multiple_replicated2; -SELECT sum(cityHash64(*)) FROM test.compression_codec_multiple_replicated2; -SELECT sum(cityHash64(*)) FROM test.compression_codec_multiple_replicated1; +SELECT sum(cityHash64(*)) FROM compression_codec_multiple_replicated2; +SELECT sum(cityHash64(*)) FROM compression_codec_multiple_replicated1; -DROP TABLE IF EXISTS test.compression_codec_multiple_replicated1; -DROP TABLE IF EXISTS test.compression_codec_multiple_replicated2; +DROP TABLE IF EXISTS compression_codec_multiple_replicated1; +DROP TABLE IF EXISTS compression_codec_multiple_replicated2; -DROP TABLE IF EXISTS test.compression_codec_multiple_more_types_replicated; +DROP TABLE IF EXISTS compression_codec_multiple_more_types_replicated; -CREATE TABLE test.compression_codec_multiple_more_types_replicated ( +CREATE TABLE compression_codec_multiple_more_types_replicated ( id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, Delta(2), Delta(4), Delta(1), LZ4HC), data FixedString(12) CODEC(ZSTD, ZSTD, Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC), ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD, Delta(8)) ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple_more_types_replicated', '1') ORDER BY tuple(); -SHOW CREATE TABLE test.compression_codec_multiple_more_types_replicated; +SHOW CREATE TABLE compression_codec_multiple_more_types_replicated; -INSERT INTO test.compression_codec_multiple_more_types_replicated VALUES(1.5555555555555, 'hello world!', [77], ['John']); -INSERT INTO test.compression_codec_multiple_more_types_replicated VALUES(7.1, 'xxxxxxxxxxxx', [127], ['Henry']); +INSERT INTO compression_codec_multiple_more_types_replicated VALUES(1.5555555555555, 'hello world!', [77], ['John']); +INSERT INTO compression_codec_multiple_more_types_replicated VALUES(7.1, 'xxxxxxxxxxxx', [127], ['Henry']); -SELECT * FROM test.compression_codec_multiple_more_types_replicated order by id; +SELECT * FROM compression_codec_multiple_more_types_replicated order by id; -DROP TABLE IF EXISTS test.compression_codec_multiple_with_key_replicated; +DROP TABLE IF EXISTS compression_codec_multiple_with_key_replicated; SET network_compression_method = 'zstd'; SET network_zstd_compression_level = 5; -CREATE TABLE test.compression_codec_multiple_with_key_replicated ( +CREATE TABLE compression_codec_multiple_with_key_replicated ( somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta), id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta), data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4) ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00910/compression_codec_multiple_with_key_replicated', '1') PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; -INSERT INTO test.compression_codec_multiple_with_key_replicated VALUES(toDate('2018-10-12'), 100000, 'hello'), (toDate('2018-10-12'), 100002, 'world'), (toDate('2018-10-12'), 1111, '!'); +INSERT INTO compression_codec_multiple_with_key_replicated VALUES(toDate('2018-10-12'), 100000, 'hello'), (toDate('2018-10-12'), 100002, 'world'), (toDate('2018-10-12'), 1111, '!'); -SELECT data FROM test.compression_codec_multiple_with_key_replicated WHERE id BETWEEN 3 AND 1112; +SELECT data FROM compression_codec_multiple_with_key_replicated WHERE id BETWEEN 3 AND 1112; -INSERT INTO test.compression_codec_multiple_with_key_replicated SELECT toDate('2018-10-12'), number, toString(number) FROM system.numbers LIMIT 1000; +INSERT INTO compression_codec_multiple_with_key_replicated SELECT toDate('2018-10-12'), number, toString(number) FROM system.numbers LIMIT 1000; -SELECT COUNT(DISTINCT data) FROM test.compression_codec_multiple_with_key_replicated WHERE id < 222; +SELECT COUNT(DISTINCT data) FROM compression_codec_multiple_with_key_replicated WHERE id < 222; -DROP TABLE IF EXISTS test.compression_codec_multiple_with_key_replicated; +DROP TABLE IF EXISTS compression_codec_multiple_with_key_replicated; +DROP TABLE compression_codec_multiple_more_types_replicated; diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference index 16818d9c7dc..e84c3beabee 100644 --- a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference +++ b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.reference @@ -1,3 +1,3 @@ 200 400 -CREATE TABLE test.ttl_repl2\n(\n `d` Date,\n `x` UInt32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00933/ttl_repl\', \'2\')\nPARTITION BY toDayOfMonth(d)\nORDER BY x\nTTL d + toIntervalDay(1)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.ttl_repl2\n(\n `d` Date,\n `x` UInt32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_00933/ttl_repl\', \'2\')\nPARTITION BY toDayOfMonth(d)\nORDER BY x\nTTL d + toIntervalDay(1)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql index ac45f2903d5..dbbbe887e9f 100644 --- a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sql @@ -1,25 +1,28 @@ -DROP TABLE IF EXISTS test.ttl_repl1; -DROP TABLE IF EXISTS test.ttl_repl2; +DROP TABLE IF EXISTS ttl_repl1; +DROP TABLE IF EXISTS ttl_repl2; -CREATE TABLE test.ttl_repl1(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00933/ttl_repl', '1') +CREATE TABLE ttl_repl1(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00933/ttl_repl', '1') PARTITION BY toDayOfMonth(d) ORDER BY x TTL d + INTERVAL 1 DAY; -CREATE TABLE test.ttl_repl2(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00933/ttl_repl', '2') +CREATE TABLE ttl_repl2(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00933/ttl_repl', '2') PARTITION BY toDayOfMonth(d) ORDER BY x TTL d + INTERVAL 1 DAY; -INSERT INTO TABLE test.ttl_repl1 VALUES (toDate('2000-10-10 00:00:00'), 100); -INSERT INTO TABLE test.ttl_repl1 VALUES (toDate('2100-10-10 00:00:00'), 200); +INSERT INTO TABLE ttl_repl1 VALUES (toDate('2000-10-10 00:00:00'), 100); +INSERT INTO TABLE ttl_repl1 VALUES (toDate('2100-10-10 00:00:00'), 200); -ALTER TABLE test.ttl_repl1 MODIFY TTL d + INTERVAL 1 DAY; -SYSTEM SYNC REPLICA test.ttl_repl2; +ALTER TABLE ttl_repl1 MODIFY TTL d + INTERVAL 1 DAY; +SYSTEM SYNC REPLICA ttl_repl2; -INSERT INTO TABLE test.ttl_repl1 VALUES (toDate('2000-10-10 00:00:00'), 300); -INSERT INTO TABLE test.ttl_repl1 VALUES (toDate('2100-10-10 00:00:00'), 400); +INSERT INTO TABLE ttl_repl1 VALUES (toDate('2000-10-10 00:00:00'), 300); +INSERT INTO TABLE ttl_repl1 VALUES (toDate('2100-10-10 00:00:00'), 400); -SYSTEM SYNC REPLICA test.ttl_repl2; +SYSTEM SYNC REPLICA ttl_repl2; SELECT sleep(1) format Null; -- wait for probable merges after inserts -OPTIMIZE TABLE test.ttl_repl2 FINAL; -SELECT x FROM test.ttl_repl2 ORDER BY x; +OPTIMIZE TABLE ttl_repl2 FINAL; +SELECT x FROM ttl_repl2 ORDER BY x; -SHOW CREATE TABLE test.ttl_repl2; +SHOW CREATE TABLE ttl_repl2; + +DROP TABLE ttl_repl1; +DROP TABLE ttl_repl2; diff --git a/tests/queries/0_stateless/00967_insert_into_distributed_different_types.sql b/tests/queries/0_stateless/00967_insert_into_distributed_different_types.sql index 33f16eb241c..455fab694cd 100644 --- a/tests/queries/0_stateless/00967_insert_into_distributed_different_types.sql +++ b/tests/queries/0_stateless/00967_insert_into_distributed_different_types.sql @@ -7,3 +7,6 @@ CREATE TABLE underlying_00967 (key Nullable(UInt64)) Engine=TinyLog(); INSERT INTO dist_00967 SELECT toUInt64(number) FROM system.numbers LIMIT 1; SELECT * FROM dist_00967; + +DROP TABLE dist_00967; +DROP TABLE underlying_00967; diff --git a/tests/queries/0_stateless/01056_create_table_as.sql b/tests/queries/0_stateless/01056_create_table_as.sql index 089daefe38d..32a39646170 100644 --- a/tests/queries/0_stateless/01056_create_table_as.sql +++ b/tests/queries/0_stateless/01056_create_table_as.sql @@ -45,3 +45,7 @@ CREATE TABLE t3 AS dict; -- { serverError 80; } DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t3; +DROP DICTIONARY dict; +DROP TABLE test_01056_dict_data.dict_data; + +DROP DATABASE test_01056_dict_data; diff --git a/tests/queries/0_stateless/01114_materialize_clear_index_compact_parts.sql b/tests/queries/0_stateless/01114_materialize_clear_index_compact_parts.sql index 404922f36bb..831cb25d967 100644 --- a/tests/queries/0_stateless/01114_materialize_clear_index_compact_parts.sql +++ b/tests/queries/0_stateless/01114_materialize_clear_index_compact_parts.sql @@ -30,3 +30,5 @@ SELECT count() FROM minmax_compact WHERE i64 = 2; -- { serverError 158 } set max_rows_to_read = 10; SELECT count() FROM minmax_compact WHERE i64 = 2; + +DROP TABLE minmax_compact; diff --git a/tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.sql b/tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.sql index 8a1601e3faa..dbc2d7c9f5d 100644 --- a/tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.sql +++ b/tests/queries/0_stateless/01142_merge_join_lc_and_nullable_in_key.sql @@ -46,3 +46,6 @@ SELECT '-'; SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x; SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x; SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x; + +DROP TABLE nr; +DROP TABLE t; diff --git a/tests/queries/0_stateless/01227_distributed_global_in_issue_2610.sql b/tests/queries/0_stateless/01227_distributed_global_in_issue_2610.sql index a063e417e3a..781e9d6f8f9 100644 --- a/tests/queries/0_stateless/01227_distributed_global_in_issue_2610.sql +++ b/tests/queries/0_stateless/01227_distributed_global_in_issue_2610.sql @@ -4,3 +4,5 @@ create table data_01227 (key Int) Engine=MergeTree() order by key; insert into data_01227 select * from numbers(10); select * from remote('127.1', currentDatabase(), data_01227) prewhere key global in (select key from data_01227 prewhere key = 2); select * from cluster('test_cluster_two_shards', currentDatabase(), data_01227) prewhere key global in (select key from data_01227 prewhere key = 2); + +drop table data_01227; diff --git a/tests/queries/0_stateless/01231_distributed_aggregation_memory_efficient_mix_levels.sql b/tests/queries/0_stateless/01231_distributed_aggregation_memory_efficient_mix_levels.sql index 31f09b35bf3..80f7625900a 100644 --- a/tests/queries/0_stateless/01231_distributed_aggregation_memory_efficient_mix_levels.sql +++ b/tests/queries/0_stateless/01231_distributed_aggregation_memory_efficient_mix_levels.sql @@ -23,3 +23,8 @@ select x, count() from ma_dist group by x order by x; drop table if exists shard_0.shard_01231_distributed_aggregation_memory_efficient; drop table if exists shard_1.shard_01231_distributed_aggregation_memory_efficient; + +drop table ma_dist; + +drop database shard_0; +drop database shard_1; diff --git a/tests/queries/0_stateless/01231_operator_null_in.sql b/tests/queries/0_stateless/01231_operator_null_in.sql index ddebaf23900..27ab0bbd838 100644 --- a/tests/queries/0_stateless/01231_operator_null_in.sql +++ b/tests/queries/0_stateless/01231_operator_null_in.sql @@ -141,3 +141,4 @@ SELECT arraySort(x -> (x.1, x.2), groupArray(t)) == [(2, NULL), (NULL, NULL)] FR SELECT arraySort(x -> (x.1, x.2), groupArray(t)) == [] FROM null_in_tuple WHERE t not in ((1, '1'), (1, NULL), (2, NULL), (NULL, '3'), (NULL, NULL)); DROP TABLE IF EXISTS null_in_tuple; +DROP TABLE test_set2; diff --git a/tests/queries/0_stateless/01232_extremes.sql b/tests/queries/0_stateless/01232_extremes.sql index e46a6602766..b0670f731c6 100644 --- a/tests/queries/0_stateless/01232_extremes.sql +++ b/tests/queries/0_stateless/01232_extremes.sql @@ -52,3 +52,5 @@ drop table if exists shard_1.num2_01232; drop table if exists distr; drop table if exists distr2; +drop database shard_0; +drop database shard_1; diff --git a/tests/queries/0_stateless/01236_distributed_over_live_view_over_distributed.sql b/tests/queries/0_stateless/01236_distributed_over_live_view_over_distributed.sql index 618bdc1c5d2..522fc5dafbe 100644 --- a/tests/queries/0_stateless/01236_distributed_over_live_view_over_distributed.sql +++ b/tests/queries/0_stateless/01236_distributed_over_live_view_over_distributed.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS visits_layer; CREATE TABLE visits (StartDate Date) ENGINE MergeTree ORDER BY(StartDate); CREATE TABLE visits_layer (StartDate Date) ENGINE Distributed(test_cluster_two_shards_localhost, currentDatabase(), 'visits', rand()); -CREATE LIVE VIEW lv AS SELECT * FROM visits_layer ORDER BY StartDate; +CREATE LIVE VIEW lv AS SELECT * FROM visits_layer ORDER BY StartDate; CREATE TABLE visits_layer_lv (StartDate Date) ENGINE Distributed(test_cluster_two_shards_localhost, currentDatabase(), 'lv', rand()); @@ -20,3 +20,4 @@ DROP TABLE visits; DROP TABLE visits_layer; DROP TABLE lv; +DROP TABLE visits_layer_lv; diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index cee9b8c9fde..f3f1905b901 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -1,17 +1,17 @@ drop table if exists test_graphite; create table test_graphite (key UInt32, Path String, Time DateTime, Value Float64, Version UInt32, col UInt64) engine = GraphiteMergeTree('graphite_rollup') order by key settings index_granularity=10; -insert into test_graphite -select 1, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +insert into test_graphite +select 1, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all select 2, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all select 1, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all select 2, 'sum_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all -select 1, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all +select 1, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all select 2, 'max_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all select 1, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all select 2, 'max_2', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300); -insert into test_graphite +insert into test_graphite select 1, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all select 2, 'sum_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all select 1, 'sum_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all @@ -24,3 +24,5 @@ select 2, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number optimize table test_graphite; select key, Path, Value, Version, col from test_graphite order by key, Path, Time desc; + +drop table test_graphite; diff --git a/tests/queries/0_stateless/01246_buffer_flush.sql b/tests/queries/0_stateless/01246_buffer_flush.sql index efe0adf703a..47891a7f00e 100644 --- a/tests/queries/0_stateless/01246_buffer_flush.sql +++ b/tests/queries/0_stateless/01246_buffer_flush.sql @@ -42,3 +42,5 @@ select 'drop'; insert into buffer_01256 select * from system.numbers limit 10; drop table if exists buffer_01256; select count() from data_01256; + +drop table data_01256; diff --git a/tests/queries/0_stateless/01246_finalize_aggregation_race.sql b/tests/queries/0_stateless/01246_finalize_aggregation_race.sql index 336fe6bcfea..c4946bd9e83 100644 --- a/tests/queries/0_stateless/01246_finalize_aggregation_race.sql +++ b/tests/queries/0_stateless/01246_finalize_aggregation_race.sql @@ -2,7 +2,7 @@ drop table if exists test_quantile; create table test_quantile (x AggregateFunction(quantileTiming(0.2), UInt64)) engine = Memory; insert into test_quantile select medianTimingState(.2)(number) from (select * from numbers(1000) order by number desc); select y from ( -select finalizeAggregation(x) as y from test_quantile union all +select finalizeAggregation(x) as y from test_quantile union all select finalizeAggregation(x) as y from test_quantile union all select finalizeAggregation(x) as y from test_quantile union all select finalizeAggregation(x) as y from test_quantile union all @@ -21,3 +21,4 @@ select finalizeAggregation(x) as y from test_quantile union all select finalizeAggregation(x) as y from test_quantile union all select finalizeAggregation(x) as y from test_quantile) order by y; +drop table test_quantile; diff --git a/tests/queries/0_stateless/01247_optimize_distributed_group_by_sharding_key_dist_on_dist.sql b/tests/queries/0_stateless/01247_optimize_distributed_group_by_sharding_key_dist_on_dist.sql index 7654ba71cc9..1000e956583 100644 --- a/tests/queries/0_stateless/01247_optimize_distributed_group_by_sharding_key_dist_on_dist.sql +++ b/tests/queries/0_stateless/01247_optimize_distributed_group_by_sharding_key_dist_on_dist.sql @@ -39,3 +39,4 @@ select count(), * from dist_01247 group by number order by number limit 1; drop table dist_01247; drop table dist_layer_01247; +drop table data_01247; diff --git a/tests/queries/0_stateless/01268_data_numeric_parameters.sql b/tests/queries/0_stateless/01268_data_numeric_parameters.sql index eceba51e7f5..3450fef9a6f 100644 --- a/tests/queries/0_stateless/01268_data_numeric_parameters.sql +++ b/tests/queries/0_stateless/01268_data_numeric_parameters.sql @@ -40,3 +40,7 @@ CREATE TABLE strings ( INSERT INTO strings VALUES ('test', 'string'); SELECT toTypeName(a), toTypeName(b) FROM strings; + +DROP TABLE floats; +DROP TABLE ints; +DROP TABLE strings; diff --git a/tests/queries/0_stateless/01268_mv_scalars.sql b/tests/queries/0_stateless/01268_mv_scalars.sql index fc082aaf88c..f6621502ca0 100644 --- a/tests/queries/0_stateless/01268_mv_scalars.sql +++ b/tests/queries/0_stateless/01268_mv_scalars.sql @@ -31,3 +31,5 @@ drop table dest_table_mv; drop table left_table; drop table right_table; drop table dest_table; +drop table src_table; +drop view dst_mv; diff --git a/tests/queries/0_stateless/01272_totals_and_filter_bug.sql b/tests/queries/0_stateless/01272_totals_and_filter_bug.sql index de751eb73bd..11992ab052a 100644 --- a/tests/queries/0_stateless/01272_totals_and_filter_bug.sql +++ b/tests/queries/0_stateless/01272_totals_and_filter_bug.sql @@ -9,22 +9,22 @@ INSERT INTO foo VALUES ('2020-01-01', 'test1', 10), ('2020-01-01', 'test2', 20); INSERT INTO bar VALUES ('2020-01-01', 'test2', 30), ('2020-01-01', 'test3', 40); SELECT - dimension_1, - sum_metric_1, + dimension_1, + sum_metric_1, sum_metric_2 -FROM +FROM ( SELECT - dimension_1, + dimension_1, sum(metric_1) AS sum_metric_1 FROM foo GROUP BY dimension_1 WITH TOTALS ) AS subquery_1 -ALL FULL OUTER JOIN +ALL FULL OUTER JOIN ( SELECT - dimension_1, + dimension_1, sum(metric_2) AS sum_metric_2 FROM bar GROUP BY dimension_1 @@ -32,3 +32,6 @@ ALL FULL OUTER JOIN ) AS subquery_2 USING (dimension_1) WHERE sum_metric_2 < 20 ORDER BY dimension_1 ASC; + +DROP TABLE foo; +DROP TABLE bar; diff --git a/tests/queries/0_stateless/01273_lc_fixed_string_field.sql b/tests/queries/0_stateless/01273_lc_fixed_string_field.sql index 18ef71fd6dc..11f93e918c6 100644 --- a/tests/queries/0_stateless/01273_lc_fixed_string_field.sql +++ b/tests/queries/0_stateless/01273_lc_fixed_string_field.sql @@ -13,3 +13,5 @@ INSERT INTO t (d, s, c) VALUES ('2020-01-01', 'ABC', 2); OPTIMIZE TABLE t; SELECT * FROM t; + +DROP TABLE t; diff --git a/tests/queries/0_stateless/01275_parallel_mv.sql b/tests/queries/0_stateless/01275_parallel_mv.sql index b67fbf02f8d..5d4dffae50a 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.sql +++ b/tests/queries/0_stateless/01275_parallel_mv.sql @@ -16,3 +16,8 @@ select count() from testX; select count() from testXA; select count() from testXB; select count() from testXC; + +drop table testX; +drop view testXA; +drop view testXB; +drop view testXC; diff --git a/tests/queries/0_stateless/01279_dist_group_by.sql b/tests/queries/0_stateless/01279_dist_group_by.sql index 207b203f5d0..331efd4b687 100644 --- a/tests/queries/0_stateless/01279_dist_group_by.sql +++ b/tests/queries/0_stateless/01279_dist_group_by.sql @@ -7,3 +7,5 @@ set max_rows_to_group_by=10; set group_by_overflow_mode='any'; set group_by_two_level_threshold=100; select * from data_01279 group by key format Null; + +drop table data_01279; diff --git a/tests/queries/0_stateless/01283_max_threads_simple_query_optimization.sql b/tests/queries/0_stateless/01283_max_threads_simple_query_optimization.sql index 8de0f40229c..a7e51047eba 100644 --- a/tests/queries/0_stateless/01283_max_threads_simple_query_optimization.sql +++ b/tests/queries/0_stateless/01283_max_threads_simple_query_optimization.sql @@ -19,3 +19,5 @@ FROM system.query_log WHERE type = 'QueryFinish' AND query LIKE '%data_01283 LIMIT 1%' GROUP BY thread_ids FORMAT Null; + +DROP TABLE data_01283; diff --git a/tests/queries/0_stateless/001283_strict_resize_bug.reference b/tests/queries/0_stateless/01283_strict_resize_bug.reference similarity index 100% rename from tests/queries/0_stateless/001283_strict_resize_bug.reference rename to tests/queries/0_stateless/01283_strict_resize_bug.reference diff --git a/tests/queries/0_stateless/001283_strict_resize_bug.sql b/tests/queries/0_stateless/01283_strict_resize_bug.sql similarity index 90% rename from tests/queries/0_stateless/001283_strict_resize_bug.sql rename to tests/queries/0_stateless/01283_strict_resize_bug.sql index f462f50c61f..ee1aef05cd2 100644 --- a/tests/queries/0_stateless/001283_strict_resize_bug.sql +++ b/tests/queries/0_stateless/01283_strict_resize_bug.sql @@ -4,4 +4,4 @@ insert into num_10m select * from numbers(10000000); select * from (select sum(number) from num_10m union all select sum(number) from num_10m) limit 1 settings max_block_size = 1024; -drop table if exists num_1m; +drop table if exists num_10m; diff --git a/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.sql b/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.sql index 110c5b65cab..575b4dfa4a5 100644 --- a/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.sql +++ b/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.sql @@ -32,3 +32,5 @@ OPTIMIZE TABLE data_01285 FINAL; SELECT * FROM data_01285; -- and this passes even without fix. SELECT * FROM data_01285 WHERE assumeNotNull(value) = 3; + +DROP TABLE data_01285; diff --git a/tests/queries/0_stateless/01291_geo_types.sql b/tests/queries/0_stateless/01291_geo_types.sql index 0d923f08ccd..6b686ddf520 100644 --- a/tests/queries/0_stateless/01291_geo_types.sql +++ b/tests/queries/0_stateless/01291_geo_types.sql @@ -7,3 +7,5 @@ CREATE TABLE geo (a Point, b Ring, c Polygon, d MultiPolygon) ENGINE=Memory(); INSERT INTO geo VALUES((0, 0), [(0, 0), (10, 0), (10, 10), (0, 10)], [[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]], [[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]); SELECT * from geo; + +DROP TABLE geo; diff --git a/tests/queries/0_stateless/01293_system_distribution_queue.sql b/tests/queries/0_stateless/01293_system_distribution_queue.sql index 4c9c690af09..8f84bbac41f 100644 --- a/tests/queries/0_stateless/01293_system_distribution_queue.sql +++ b/tests/queries/0_stateless/01293_system_distribution_queue.sql @@ -24,3 +24,6 @@ select is_blocked, error_count, data_files, data_compressed_bytes from system.di select 'UNBLOCK'; system start distributed sends dist_01293; select is_blocked, error_count, data_files, data_compressed_bytes from system.distribution_queue; + +drop table null_01293; +drop table dist_01293; diff --git a/tests/queries/0_stateless/01296_pipeline_stuck.sql b/tests/queries/0_stateless/01296_pipeline_stuck.sql index eeb67362634..2a23e6a9bf8 100644 --- a/tests/queries/0_stateless/01296_pipeline_stuck.sql +++ b/tests/queries/0_stateless/01296_pipeline_stuck.sql @@ -16,3 +16,5 @@ select 'INSERT SELECT max_insert_threads max_threads'; set max_insert_threads=2; insert into data_01295 select * from data_01295 final settings max_threads=2; -- no stuck for now select * from data_01295; + +drop table data_01295; diff --git a/tests/queries/0_stateless/01318_decrypt.sql b/tests/queries/0_stateless/01318_decrypt.sql index 796c42db1ab..fecca593272 100644 --- a/tests/queries/0_stateless/01318_decrypt.sql +++ b/tests/queries/0_stateless/01318_decrypt.sql @@ -150,3 +150,5 @@ WITH SELECT hex(decrypt('aes-256-gcm', concat(ciphertext, tag), key, iv, aad)) as plaintext_actual, plaintext_actual = hex(plaintext); + +DROP TABLE encryption_test; diff --git a/tests/queries/0_stateless/01318_encrypt.sql b/tests/queries/0_stateless/01318_encrypt.sql index 9766988764a..a9a60c89836 100644 --- a/tests/queries/0_stateless/01318_encrypt.sql +++ b/tests/queries/0_stateless/01318_encrypt.sql @@ -154,3 +154,5 @@ WITH SELECT hex(encrypt('aes-256-gcm', plaintext, key, iv, aad)) as ciphertext_actual, ciphertext_actual = concat(hex(ciphertext), hex(tag)); + +DROP TABLE encryption_test; diff --git a/tests/queries/0_stateless/01319_mv_constants_bug.sql b/tests/queries/0_stateless/01319_mv_constants_bug.sql index 191183ab286..4abb9d61b6c 100644 --- a/tests/queries/0_stateless/01319_mv_constants_bug.sql +++ b/tests/queries/0_stateless/01319_mv_constants_bug.sql @@ -22,3 +22,4 @@ DROP TABLE IF EXISTS distributed_table_1; DROP TABLE IF EXISTS distributed_table_2; DROP TABLE IF EXISTS local_table_1; DROP TABLE IF EXISTS local_table_2; +DROP TABLE local_table_merged; diff --git a/tests/queries/0_stateless/01319_optimize_skip_unused_shards_nesting.sql b/tests/queries/0_stateless/01319_optimize_skip_unused_shards_nesting.sql index b8a48c27e5f..4f7d6b81002 100644 --- a/tests/queries/0_stateless/01319_optimize_skip_unused_shards_nesting.sql +++ b/tests/queries/0_stateless/01319_optimize_skip_unused_shards_nesting.sql @@ -20,3 +20,7 @@ select * from dist_01319 where key = 1; set force_optimize_skip_unused_shards_nesting=2; set optimize_skip_unused_shards_nesting=1; select * from dist_01319 where key = 1; + +drop table data_01319; +drop table dist_01319; +drop table dist_layer_01319; diff --git a/tests/queries/0_stateless/01320_optimize_skip_unused_shards_no_non_deterministic.sql b/tests/queries/0_stateless/01320_optimize_skip_unused_shards_no_non_deterministic.sql index ca58f7be94c..f827bb733b4 100644 --- a/tests/queries/0_stateless/01320_optimize_skip_unused_shards_no_non_deterministic.sql +++ b/tests/queries/0_stateless/01320_optimize_skip_unused_shards_no_non_deterministic.sql @@ -8,3 +8,6 @@ create table dist_01320 as data_01320 Engine=Distributed(test_cluster_two_shards set optimize_skip_unused_shards=1; set force_optimize_skip_unused_shards=1; select * from dist_01320 where key = 0; -- { serverError 507 } + +drop table data_01320; +drop table dist_01320; diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql index 0719282734a..d185973f564 100644 --- a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql +++ b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper.sql @@ -47,3 +47,4 @@ ALTER TABLE test DROP COLUMN x; -- { serverError 47 } ALTER TABLE test DROP COLUMN y; -- { serverError 47 } DROP TABLE test; +DROP TABLE test2; diff --git a/tests/queries/0_stateless/01392_column_resolve.sql b/tests/queries/0_stateless/01392_column_resolve.sql index 93d84359e1f..728ccfd0728 100644 --- a/tests/queries/0_stateless/01392_column_resolve.sql +++ b/tests/queries/0_stateless/01392_column_resolve.sql @@ -1,15 +1,14 @@ -DROP TABLE IF EXISTS tableConversion; -DROP TABLE IF EXISTS tableClick; -DROP TABLE IF EXISTS leftjoin; +DROP DATABASE IF EXISTS test_01392; +CREATE DATABASE test_01392; -CREATE TABLE default.tableConversion (conversionId String, value Nullable(Double)) ENGINE = Log(); -CREATE TABLE default.tableClick (clickId String, conversionId String, value Nullable(Double)) ENGINE = Log(); -CREATE TABLE default.leftjoin (id String) ENGINE = Log(); +CREATE TABLE test_01392.tableConversion (conversionId String, value Nullable(Double)) ENGINE = Log(); +CREATE TABLE test_01392.tableClick (clickId String, conversionId String, value Nullable(Double)) ENGINE = Log(); +CREATE TABLE test_01392.leftjoin (id String) ENGINE = Log(); -INSERT INTO default.tableConversion(conversionId, value) VALUES ('Conversion 1', 1); -INSERT INTO default.tableClick(clickId, conversionId, value) VALUES ('Click 1', 'Conversion 1', 14); -INSERT INTO default.tableClick(clickId, conversionId, value) VALUES ('Click 2', 'Conversion 1', 15); -INSERT INTO default.tableClick(clickId, conversionId, value) VALUES ('Click 3', 'Conversion 1', 16); +INSERT INTO test_01392.tableConversion(conversionId, value) VALUES ('Conversion 1', 1); +INSERT INTO test_01392.tableClick(clickId, conversionId, value) VALUES ('Click 1', 'Conversion 1', 14); +INSERT INTO test_01392.tableClick(clickId, conversionId, value) VALUES ('Click 2', 'Conversion 1', 15); +INSERT INTO test_01392.tableClick(clickId, conversionId, value) VALUES ('Click 3', 'Conversion 1', 16); SELECT conversion.conversionId AS myConversionId, @@ -17,17 +16,19 @@ SELECT click.myValue AS myValue FROM ( SELECT conversionId, value as myValue - FROM default.tableConversion + FROM test_01392.tableConversion ) AS conversion INNER JOIN ( SELECT clickId, conversionId, value as myValue - FROM default.tableClick + FROM test_01392.tableClick ) AS click ON click.conversionId = conversion.conversionId LEFT JOIN ( - SELECT * FROM default.leftjoin + SELECT * FROM test_01392.leftjoin ) AS dummy ON (dummy.id = conversion.conversionId) ORDER BY myValue; -DROP TABLE IF EXISTS tableConversion; -DROP TABLE IF EXISTS tableClick; -DROP TABLE IF EXISTS leftjoin; +DROP TABLE test_01392.tableConversion; +DROP TABLE test_01392.tableClick; +DROP TABLE test_01392.leftjoin; + +DROP DATABASE test_01392; diff --git a/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql index 8a19865359b..8f83e1c15dd 100644 --- a/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql @@ -13,3 +13,4 @@ INSERT INTO test_lc VALUES ('ab', '1', 0.1), ('ab', '2', 0.2), ('cd', '3', 0.3); SELECT joinGet(test_lc, 'c', 'ab', '1'); DROP TABLE test_joinGet; +DROP TABLE test_lc; diff --git a/tests/queries/0_stateless/01409_topK_merge.sql b/tests/queries/0_stateless/01409_topK_merge.sql index 5ac7c350093..d7659351540 100644 --- a/tests/queries/0_stateless/01409_topK_merge.sql +++ b/tests/queries/0_stateless/01409_topK_merge.sql @@ -11,3 +11,5 @@ select length(topKWeighted(20)(number, 1)) from remote('127.{1,1}', currentDatab select 'AggregateFunctionTopKGenericData'; select length(topK(20)((number, ''))) from remote('127.{1,1}', currentDatabase(), data_01409); select length(topKWeighted(20)((number, ''), 1)) from remote('127.{1,1}', currentDatabase(), data_01409); + +drop table data_01409; diff --git a/tests/queries/0_stateless/01413_truncate_without_table_keyword.sql b/tests/queries/0_stateless/01413_truncate_without_table_keyword.sql index 266c5b84be3..c6819f77e80 100644 --- a/tests/queries/0_stateless/01413_truncate_without_table_keyword.sql +++ b/tests/queries/0_stateless/01413_truncate_without_table_keyword.sql @@ -10,3 +10,4 @@ TRUNCATE truncate_test; SELECT * FROM truncate_test ORDER BY uint8; +DROP TABLE truncate_test; diff --git a/tests/queries/0_stateless/01419_skip_index_compact_parts.sql b/tests/queries/0_stateless/01419_skip_index_compact_parts.sql index 8b03f28df44..580fcf7f41e 100644 --- a/tests/queries/0_stateless/01419_skip_index_compact_parts.sql +++ b/tests/queries/0_stateless/01419_skip_index_compact_parts.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS index_compact; - + CREATE TABLE index_compact(a UInt32, b UInt32, index i1 b type minmax granularity 1) ENGINE = MergeTree ORDER BY a SETTINGS min_rows_for_wide_part = 1000, index_granularity = 128, merge_max_block_size = 100; @@ -10,3 +10,5 @@ INSERT INTO index_compact SELECT number, toString(number) FROM numbers(30); OPTIMIZE TABLE index_compact FINAL; SELECT count() FROM index_compact WHERE b < 10; + +DROP TABLE index_compact; diff --git a/tests/queries/0_stateless/01456_low_cardinality_sorting_bugfix.sql b/tests/queries/0_stateless/01456_low_cardinality_sorting_bugfix.sql index 507a798e7b6..06412d4b062 100644 --- a/tests/queries/0_stateless/01456_low_cardinality_sorting_bugfix.sql +++ b/tests/queries/0_stateless/01456_low_cardinality_sorting_bugfix.sql @@ -39,3 +39,5 @@ SELECT cast(color,'String') color, timestamp FROM order_test1 GROUP BY color, timestamp ORDER BY color ASC, timestamp DESC; + +DROP TABLE order_test1; diff --git a/tests/queries/0_stateless/01457_create_as_table_function_structure.sql b/tests/queries/0_stateless/01457_create_as_table_function_structure.sql index 1bb0d9c54ca..1c9c1e1ef44 100644 --- a/tests/queries/0_stateless/01457_create_as_table_function_structure.sql +++ b/tests/queries/0_stateless/01457_create_as_table_function_structure.sql @@ -32,3 +32,4 @@ SELECT (*,).1 AS c, toTypeName(c) FROM test_01457.tf_merge; DROP DATABASE test_01457; +DROP TABLE tmp; diff --git a/tests/queries/0_stateless/01460_DistributedFilesToInsert.sql b/tests/queries/0_stateless/01460_DistributedFilesToInsert.sql index 8ce10cf133f..34c0d55d573 100644 --- a/tests/queries/0_stateless/01460_DistributedFilesToInsert.sql +++ b/tests/queries/0_stateless/01460_DistributedFilesToInsert.sql @@ -40,3 +40,5 @@ select sleep(1) format Null; -- distributed_directory_monitor_sleep_time_ms select value from system.metrics where metric = 'DistributedFilesToInsert'; drop table dist_01460; select value from system.metrics where metric = 'DistributedFilesToInsert'; + +drop table data_01460; diff --git a/tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql b/tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql index ab316a377fd..04276087ece 100644 --- a/tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql +++ b/tests/queries/0_stateless/01463_test_alter_live_view_refresh.sql @@ -8,3 +8,6 @@ CREATE LIVE VIEW live1 AS SELECT * FROM test0; select 'ALTER LIVE VIEW live1 REFRESH'; ALTER LIVE VIEW live1 REFRESH; -- success + +DROP TABLE test0; +DROP VIEW live1; diff --git a/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql b/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql index f6f7471711a..39e04804bfb 100644 --- a/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql +++ b/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql @@ -34,3 +34,4 @@ DROP TABLE IF EXISTS t; DROP DATABASE shard_0; DROP DATABASE shard_1; +DROP DATABASE main_01487; diff --git a/tests/queries/0_stateless/01511_prewhere_with_virtuals.sql b/tests/queries/0_stateless/01511_prewhere_with_virtuals.sql index 8b272c8cff1..43f003fc80f 100644 --- a/tests/queries/0_stateless/01511_prewhere_with_virtuals.sql +++ b/tests/queries/0_stateless/01511_prewhere_with_virtuals.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS test_not_found_column_nothing; -CREATE TABLE test_not_found_column_nothing +CREATE TABLE test_not_found_column_nothing ( col001 UInt8, col002 UInt8 @@ -10,3 +10,5 @@ INSERT INTO test_not_found_column_nothing(col001) SELECT number FROM numbers(11) SELECT _part, count() FROM test_not_found_column_nothing PREWHERE col001 % 3 != 0 GROUP BY _part ORDER BY _part; SELECT _part FROM test_not_found_column_nothing PREWHERE col001 = 0; + +DROP TABLE test_not_found_column_nothing; diff --git a/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory.sql b/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory.sql index 38920262fba..6aa38a914f7 100644 --- a/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory.sql +++ b/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory.sql @@ -14,3 +14,5 @@ select key, groupArray(repeat('a', 200)), count() from data_01513 group by key f select key, groupArray(repeat('a', 200)), count() from data_01513 group by key format Null settings optimize_aggregation_in_order=1; -- for WITH TOTALS previous groups should be kept. select key, groupArray(repeat('a', 200)), count() from data_01513 group by key with totals format Null settings optimize_aggregation_in_order=1; -- { serverError 241; } + +drop table data_01513; diff --git a/tests/queries/0_stateless/01514_empty_buffer_different_types.sql b/tests/queries/0_stateless/01514_empty_buffer_different_types.sql index 3afadbcd33f..a1debbf7e13 100644 --- a/tests/queries/0_stateless/01514_empty_buffer_different_types.sql +++ b/tests/queries/0_stateless/01514_empty_buffer_different_types.sql @@ -10,3 +10,4 @@ select s from buffer_table1 where x = 1; select s from buffer_table1 where x = 2; DROP TABLE IF EXISTS merge_tree_table1; +DROP TABLE buffer_table1; diff --git a/tests/queries/0_stateless/01515_force_data_skipping_indices.sql b/tests/queries/0_stateless/01515_force_data_skipping_indices.sql index 5e45018707d..53d3e5c736f 100644 --- a/tests/queries/0_stateless/01515_force_data_skipping_indices.sql +++ b/tests/queries/0_stateless/01515_force_data_skipping_indices.sql @@ -29,3 +29,5 @@ SELECT * FROM data_01515 WHERE d1 = 0 SETTINGS force_data_skipping_indices=' d1 SELECT * FROM data_01515 WHERE d1_null = 0 SETTINGS force_data_skipping_indices='d1_null_idx'; -- { serverError 277 } SELECT * FROM data_01515 WHERE assumeNotNull(d1_null) = 0 SETTINGS force_data_skipping_indices='d1_null_idx'; + +DROP TABLE data_01515; diff --git a/tests/queries/0_stateless/01515_mv_and_array_join_optimisation_bag.sql b/tests/queries/0_stateless/01515_mv_and_array_join_optimisation_bag.sql index 9cca5e85da7..ad762ea65fb 100644 --- a/tests/queries/0_stateless/01515_mv_and_array_join_optimisation_bag.sql +++ b/tests/queries/0_stateless/01515_mv_and_array_join_optimisation_bag.sql @@ -24,14 +24,14 @@ SELECT sumIfState(Sign, _uniq = 1) AS Visits, sumState(Sign) AS GoalReaches FROM visits -ARRAY JOIN +ARRAY JOIN GoalsID AS GoalID, arrayEnumerateUniq(GoalsID) AS _uniq -GROUP BY +GROUP BY CounterID, StartDate, GoalID -ORDER BY +ORDER BY CounterID ASC, StartDate ASC, GoalID ASC; @@ -46,3 +46,7 @@ CREATE TABLE goal ) ENGINE = AggregatingMergeTree PARTITION BY toStartOfMonth(StartDate) ORDER BY (CounterID, StartDate, GoalID) SETTINGS index_granularity = 256; INSERT INTO visits (`CounterID`,`StartDate`,`StartTime`,`Sign`,`GoalsID`) VALUES (1, toDate('2000-01-01'), toDateTime(toDate('2000-01-01')), 1, [1]); + +DROP TABLE goal; +DROP TABLE goal_view; +DROP TABLE visits; diff --git a/tests/queries/0_stateless/01517_drop_mv_with_inner_table.sql b/tests/queries/0_stateless/01517_drop_mv_with_inner_table.sql index bbf7f0d04be..b97480c2911 100644 --- a/tests/queries/0_stateless/01517_drop_mv_with_inner_table.sql +++ b/tests/queries/0_stateless/01517_drop_mv_with_inner_table.sql @@ -38,3 +38,10 @@ create materialized view db_01517_ordinary.mv engine=Null as select * from db_01 -- drops it and hangs with Atomic engine, due to recursive DROP drop table db_01517_ordinary.mv sync; show tables from db_01517_ordinary; + +drop table db_01517_atomic_sync.source; +drop table db_01517_ordinary.source; + +drop database db_01517_atomic; +drop database db_01517_atomic_sync; +drop database db_01517_ordinary; diff --git a/tests/queries/0_stateless/01526_initial_query_id.sh b/tests/queries/0_stateless/01526_initial_query_id.sh index c5459625023..e28f9ee1e40 100755 --- a/tests/queries/0_stateless/01526_initial_query_id.sh +++ b/tests/queries/0_stateless/01526_initial_query_id.sh @@ -10,7 +10,7 @@ ${CLICKHOUSE_CLIENT} -q "select 1 format Null" "--query_id=$query_id" ${CLICKHOUSE_CURL} \ --header "X-ClickHouse-Query-Id: $query_id" \ - "http://localhost:8123/" \ + $CLICKHOUSE_URL \ --get \ --data-urlencode "query=select 1 format Null" diff --git a/tests/queries/0_stateless/01528_allow_nondeterministic_optimize_skip_unused_shards.sql b/tests/queries/0_stateless/01528_allow_nondeterministic_optimize_skip_unused_shards.sql index b0bfb2aae3f..51970db73cb 100644 --- a/tests/queries/0_stateless/01528_allow_nondeterministic_optimize_skip_unused_shards.sql +++ b/tests/queries/0_stateless/01528_allow_nondeterministic_optimize_skip_unused_shards.sql @@ -5,3 +5,5 @@ set optimize_skip_unused_shards=1; set force_optimize_skip_unused_shards=1; select * from dist_01528 where dummy = 2; -- { serverError 507; } select * from dist_01528 where dummy = 2 settings allow_nondeterministic_optimize_skip_unused_shards=1; + +drop table dist_01528; diff --git a/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql b/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql index 010b8931448..d5fb25a9241 100644 --- a/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql +++ b/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql @@ -28,7 +28,7 @@ drop database db_01530_atomic; create database db_01530_atomic Engine=Atomic; create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/db_01530_atomic/data', 'test') order by key; -- { serverError 253; } --- TODO: SYSTEM FORCE DROP and uncomment the line below to cleanup the data after test --- (otherwise the test is not retriable...) --- --- drop database db_01530_atomic sync; + +set database_atomic_wait_for_drop_and_detach_synchronously=1; + +drop database db_01530_atomic sync; diff --git a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql index 831a7282861..a71c9f9a714 100644 --- a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql +++ b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql @@ -14,3 +14,5 @@ SET merge_tree_min_rows_for_concurrent_read=10000; SET optimize_aggregation_in_order=1; SET read_in_order_two_level_merge_threshold=1; EXPLAIN PIPELINE SELECT key FROM data_01551 GROUP BY key, key/2; + +DROP TABLE data_01551; diff --git a/tests/queries/conftest.py b/tests/queries/conftest.py index 85b7250af0c..2f19ae7c479 100644 --- a/tests/queries/conftest.py +++ b/tests/queries/conftest.py @@ -36,6 +36,11 @@ def sql_query(request): return os.path.join(QUERIES_PATH, os.path.splitext(request.param)[0]) +@pytest.fixture(scope='module', params=[f for f in os.listdir(QUERIES_PATH) if f.endswith('.sh')]) +def shell_query(request): + return os.path.join(QUERIES_PATH, os.path.splitext(request.param)[0]) + + @pytest.fixture def standalone_server(bin_prefix, tmp_path): server = ServerThread(bin_prefix, str(tmp_path)) diff --git a/tests/queries/query_test.py b/tests/queries/query_test.py index d0fc5759667..adaf9bc4e5e 100644 --- a/tests/queries/query_test.py +++ b/tests/queries/query_test.py @@ -8,18 +8,16 @@ import subprocess import sys -def run_client(bin_prefix, port, query, reference, replace_map={}): - client = subprocess.Popen([bin_prefix + '-client', '--port', str(port), '-m', '-n', '--testmode'], - stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.PIPE) - result, error = client.communicate(query.encode('utf-8')) - assert client.returncode is not None, "Client should exit after processing all queries" - +def check_result(result, error, return_code, reference, replace_map): for old, new in replace_map.items(): result = result.replace(old.encode('utf-8'), new.encode('utf-8')) - if client.returncode != 0: - print(error.decode('utf-8'), file=sys.stderr) - pytest.fail('Client died unexpectedly with code {code}'.format(code=client.returncode), pytrace=False) + if return_code != 0: + try: + print(error.decode('utf-8'), file=sys.stderr) + except UnicodeDecodeError: + print(error.decode('latin1'), file=sys.stderr) # encoding with 1 symbol per 1 byte, covering all values + pytest.fail('Client died unexpectedly with code {code}'.format(code=return_code), pytrace=False) elif result != reference: pytest.fail("Query output doesn't match reference:{eol}{diff}".format( eol=os.linesep, @@ -29,13 +27,38 @@ def run_client(bin_prefix, port, query, reference, replace_map={}): pytrace=False) +def run_client(bin_prefix, port, query, reference, replace_map={}): + # We can't use `text=True` since some tests may return binary data + client = subprocess.Popen([bin_prefix + '-client', '--port', str(port), '-m', '-n', '--testmode'], + stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.PIPE) + result, error = client.communicate(query.encode('utf-8')) + assert client.returncode is not None, "Client should exit after processing all queries" + + check_result(result, error, client.returncode, reference, replace_map) + + +def run_shell(bin_prefix, tmp_dir, tcp_port, http_port, inter_port, database, path, reference, replace_map={}): + env = { + 'CLICKHOUSE_BINARY': bin_prefix, + 'CLICKHOUSE_DATABASE': database, + 'CLICKHOUSE_PORT_TCP': str(tcp_port), + 'CLICKHOUSE_PORT_HTTP': str(http_port), + 'CLICKHOUSE_PORT_INTERSERVER': str(inter_port), + 'CLICKHOUSE_TMP': tmp_dir, + } + shell = subprocess.Popen([path], env=env, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE) + result, error = shell.communicate() + assert shell.returncode is not None, "Script should exit after executing all commands" + + check_result(result, error, shell.returncode, reference, replace_map) + + def random_str(length=10): alphabet = string.ascii_lowercase + string.digits return ''.join(random.choice(alphabet) for _ in range(length)) -@pytest.mark.timeout(timeout=30, method='signal') -def test_query(bin_prefix, sql_query, standalone_server): +def test_sql_query(bin_prefix, sql_query, standalone_server): tcp_port = standalone_server.tcp_port query_path = sql_query + ".sql" @@ -56,7 +79,39 @@ def test_query(bin_prefix, sql_query, standalone_server): query = "SELECT 'SHOW ORPHANED TABLES'; SELECT name FROM system.tables WHERE database != 'system' ORDER BY (database, name);" run_client(bin_prefix, tcp_port, query, b'SHOW ORPHANED TABLES\n') - run_client(bin_prefix, tcp_port, 'DROP DATABASE {random};'.format(random=random_name), b'') + query = 'DROP DATABASE {random};'.format(random=random_name) + run_client(bin_prefix, tcp_port, query, b'') + + query = "SELECT 'SHOW ORPHANED DATABASES'; SHOW DATABASES;" + run_client(bin_prefix, tcp_port, query, b'SHOW ORPHANED DATABASES\n_temporary_and_external_tables\ndefault\nsystem\n') + + +def test_shell_query(bin_prefix, shell_query, standalone_server): + tcp_port = standalone_server.tcp_port + http_port = standalone_server.http_port + inter_port = standalone_server.inter_port + tmp_path = standalone_server.tmp_dir + + shell_path = shell_query + ".sh" + reference_path = shell_query + ".reference" + + if not os.path.exists(reference_path): + pytest.skip('No .reference file found') + + with open(reference_path, 'rb') as file: + reference = file.read() + + random_name = 'test_{random}'.format(random=random_str()) + query = 'CREATE DATABASE {random};'.format(random=random_name) + run_client(bin_prefix, tcp_port, query, b'') + + run_shell(bin_prefix, tmp_path, tcp_port, http_port, inter_port, random_name, shell_path, reference, {random_name: 'default'}) + + query = "SELECT 'SHOW ORPHANED TABLES'; SELECT name FROM system.tables WHERE database != 'system' ORDER BY (database, name);" + run_client(bin_prefix, tcp_port, query, b'SHOW ORPHANED TABLES\n') + + query = 'DROP DATABASE {random};'.format(random=random_name) + run_client(bin_prefix, tcp_port, query, b'') query = "SELECT 'SHOW ORPHANED DATABASES'; SHOW DATABASES;" run_client(bin_prefix, tcp_port, query, b'SHOW ORPHANED DATABASES\n_temporary_and_external_tables\ndefault\nsystem\n') diff --git a/tests/queries/server.py b/tests/queries/server.py index c4f8968e08a..b3c3ece0b4c 100644 --- a/tests/queries/server.py +++ b/tests/queries/server.py @@ -23,6 +23,7 @@ class ServerThread(threading.Thread): self.etc_dir = os.path.join(tmp_dir, 'etc') self.server_config = os.path.join(self.etc_dir, 'server-config.xml') self.users_config = os.path.join(self.etc_dir, 'users.xml') + self.dicts_config = os.path.join(self.etc_dir, 'dictionaries.xml') os.makedirs(self.log_dir) os.makedirs(self.etc_dir) @@ -51,6 +52,9 @@ class ServerThread(threading.Thread): with open(self.users_config, 'w') as f: f.write(ServerThread.DEFAULT_USERS_CONFIG) + with open(self.dicts_config, 'w') as f: + f.write(ServerThread.DEFAULT_DICTIONARIES_CONFIG.format(tcp_port=self.tcp_port)) + def run(self): retries = ServerThread.DEFAULT_RETRIES @@ -122,10 +126,45 @@ ServerThread.DEFAULT_SERVER_CONFIG = \ {tmp_dir}/tmp/ {tmp_dir}/data/access/ users.xml + dictionaries.xml 5368709120 + 3 Europe/Moscow + + Hello, world! + s1 + r1 + + + + Version + + sum + sum + + 0 + 600 + + + 172800 + 6000 + + + + max + + 0 + 600 + + + 172800 + 6000 + + + + @@ -180,7 +219,78 @@ ServerThread.DEFAULT_SERVER_CONFIG = \ + + + + + shard_0 + localhost + {tcp_port} + + + + + shard_1 + localhost + {tcp_port} + + + + + + + true + + 127.0.0.1 + {tcp_port} + + + + true + + 127.0.0.2 + {tcp_port} + + + + + + + + memory + + + + + + testkeeper + + + + system + part_log
+
+ + + system + query_log
+
+ + + system + query_thread_log
+
+ + + system + text_log
+
+ + + system + trace_log
+
""" @@ -242,3 +352,735 @@ ServerThread.DEFAULT_USERS_CONFIG = \ """ + + +ServerThread.DEFAULT_DICTIONARIES_CONFIG = \ +"""\ + + + flat_ints + + + localhost + {tcp_port} + default + + system + ints
+
+ + 0 + + + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + hashed_ints + + + localhost + {tcp_port} + default + + system + ints
+
+ + 0 + + + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + hashed_sparse_ints + + + localhost + {tcp_port} + default + + system + ints
+
+ + 0 + + + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + cache_ints + + + localhost + {tcp_port} + default + + system + ints
+
+ + 0 + + 1000 + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + complex_hashed_ints + + + localhost + {tcp_port} + default + + system + ints
+
+ + 0 + + + + + + + key + UInt64 + + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + complex_cache_ints + + + localhost + {tcp_port} + default + + system + ints
+
+ + 0 + + 1000 + + + + + key + UInt64 + + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ + + flat_strings + + + localhost + {tcp_port} + default + + system + strings
+
+ + 0 + + + + + + key + + + str + String + + + +
+ + + hashed_strings + + + localhost + {tcp_port} + default + + system + strings
+
+ + 0 + + + + + + key + + + str + String + + + +
+ + + cache_strings + + + localhost + {tcp_port} + default + + system + strings
+
+ + 0 + + 1000 + + + + key + + + str + String + + + +
+ + + complex_hashed_strings + + + localhost + {tcp_port} + default + + system + strings
+
+ + 0 + + + + + + + key + UInt64 + + + + str + String + + + +
+ + + complex_cache_strings + + + localhost + {tcp_port} + default + + system + strings
+
+ + 0 + + 1000 + + + + + key + UInt64 + + + + str + String + + + +
+ + + flat_decimals + + + localhost + {tcp_port} + default + + system + decimals
+
+ + 0 + + + + + + key + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + hashed_decimals + + + localhost + {tcp_port} + default + + system + decimals
+
+ + 0 + + + + + + key + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + cache_decimals + + + localhost + {tcp_port} + default + + system + decimals
+
+ + 0 + + 1000 + + + + key + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + complex_hashed_decimals + + + localhost + {tcp_port} + default + + system + decimals
+
+ + 0 + + + + + + + key + UInt64 + + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+ + + complex_cache_decimals + + + localhost + {tcp_port} + default + + system + decimals
+
+ + 0 + + 1000 + + + + + key + UInt64 + + + + d32 + Decimal32(4) + 0 + + + d64 + Decimal64(6) + 0 + + + d128 + Decimal128(1) + 0 + + +
+
+""" diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index 0b5b0940cd7..9cfec2c2a28 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -1,5 +1,8 @@ #!/usr/bin/env bash +# Don't check for ODR violation, since we may test shared build with ASAN +export ASAN_OPTIONS=detect_odr_violation=0 + export CLICKHOUSE_DATABASE=${CLICKHOUSE_DATABASE:="test"} export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL:="warning"} [ -v CLICKHOUSE_CONFIG_CLIENT ] && CLICKHOUSE_CLIENT_OPT0+=" --config-file=${CLICKHOUSE_CONFIG_CLIENT} " From f1d721fa74ac0b4ae54dd231c5c6be69812106a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 11 Nov 2020 19:58:31 +0300 Subject: [PATCH 88/94] Marked some perf test queries as short --- tests/performance/push_down_limit.xml | 2 +- tests/performance/quantile_merge.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/push_down_limit.xml b/tests/performance/push_down_limit.xml index 6ae63b54ec6..0611216410d 100644 --- a/tests/performance/push_down_limit.xml +++ b/tests/performance/push_down_limit.xml @@ -4,5 +4,5 @@ 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 + select number from numbers_view limit 100 diff --git a/tests/performance/quantile_merge.xml b/tests/performance/quantile_merge.xml index 0ddb688d8eb..ad0f0571682 100644 --- a/tests/performance/quantile_merge.xml +++ b/tests/performance/quantile_merge.xml @@ -1,3 +1,3 @@ - SELECT quantileMerge(arrayJoin(arrayMap(x -> state, range(500000)))) FROM (SELECT quantileState(rand()) AS state FROM numbers(10000)) + SELECT quantileMerge(arrayJoin(arrayMap(x -> state, range(500000)))) FROM (SELECT quantileState(rand()) AS state FROM numbers(10000)) From d7e2276a9e4a18fbd7e62341d3b6d39c21b68ec6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 11 Nov 2020 20:39:16 +0300 Subject: [PATCH 89/94] Update version_date.tsv after release 20.11.2.1 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0e55de496b7..47f6aefe406 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v20.11.2.1-stable 2020-11-11 v20.10.3.30-stable 2020-10-29 v20.10.2.20-stable 2020-10-23 v20.9.4.76-stable 2020-10-29 From f6b7bd88468ff55810cb1ac50a0045de7e86cde8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 12 Nov 2020 01:51:33 +0300 Subject: [PATCH 90/94] Add changelog for 20.11 --- CHANGELOG.md | 119 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 119 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 457346aff9a..e552fe06962 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,122 @@ +### ClickHouse release v20.11.2.1, 2020-11-11 + +#### Backward Incompatible Change + +* If some `profile` was specified in `distributed_ddl` config section, then this profile could overwrite settings of `default` profile on server startup. It's fixed, now settings of distributed DDL queries should not affect global server settings. [#16635](https://github.com/ClickHouse/ClickHouse/pull/16635) ([tavplubix](https://github.com/tavplubix)). +* Restrict to use of non-comparable data types (like `AggregateFunction`) in keys (Sorting key, Primary key, Partition key, and so on). [#16601](https://github.com/ClickHouse/ClickHouse/pull/16601) ([alesapin](https://github.com/alesapin)). +* Remove `ANALYZE` and `AST` queries, and make the setting `enable_debug_queries` obsolete since now it is the part of full featured `EXPLAIN` query. [#16536](https://github.com/ClickHouse/ClickHouse/pull/16536) ([Ivan](https://github.com/abyss7)). +* Aggregate functions `boundingRatio`, `rankCorr`, `retention`, `timeSeriesGroupSum`, `timeSeriesGroupRateSum`, `windowFunnel` were erroneously made case-insensitive. Now their names are made case sensitive as designed. Only functions that are specified in SQL standard or made for compatibility with other DBMS or functions similar to those should be case-insensitive. [#16407](https://github.com/ClickHouse/ClickHouse/pull/16407) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make `rankCorr` function return nan on insufficient data https://github.com/ClickHouse/ClickHouse/issues/16124. [#16135](https://github.com/ClickHouse/ClickHouse/pull/16135) ([hexiaoting](https://github.com/hexiaoting)). + +#### New Feature + +* Added support of LDAP as a user directory for locally non-existent users. [#12736](https://github.com/ClickHouse/ClickHouse/pull/12736) ([Denis Glazachev](https://github.com/traceon)). +* Add `system.replicated_fetches` table which shows currently running background fetches. [#16428](https://github.com/ClickHouse/ClickHouse/pull/16428) ([alesapin](https://github.com/alesapin)). +* Added setting `date_time_output_format`. [#15845](https://github.com/ClickHouse/ClickHouse/pull/15845) ([Maksim Kita](https://github.com/kitaisreal)). +* Added minimal web UI to ClickHouse. [#16158](https://github.com/ClickHouse/ClickHouse/pull/16158) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allows to read/write Single protobuf message at once (w/o length-delimiters). [#15199](https://github.com/ClickHouse/ClickHouse/pull/15199) ([filimonov](https://github.com/filimonov)). +* Added initial OpenTelemetry support. ClickHouse now accepts OpenTelemetry traceparent headers over Native and HTTP protocols, and passes them downstream in some cases. The trace spans for executed queries are saved into the `system.opentelemetry_span_log` table. [#14195](https://github.com/ClickHouse/ClickHouse/pull/14195) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Allow specify primary key in column list of `CREATE TABLE` query. This is needed for compatibility with other SQL dialects. [#15823](https://github.com/ClickHouse/ClickHouse/pull/15823) ([Maksim Kita](https://github.com/kitaisreal)). +* Implement `OFFSET offset_row_count {ROW | ROWS} FETCH {FIRST | NEXT} fetch_row_count {ROW | ROWS} {ONLY | WITH TIES}` in SELECT query with ORDER BY. This is the SQL-standard way to specify `LIMIT`. [#15855](https://github.com/ClickHouse/ClickHouse/pull/15855) ([hexiaoting](https://github.com/hexiaoting)). +* `errorCodeToName` function - return variable name of the error (useful for analyzing query_log and similar). `system.errors` table - shows how many times errors has been happened (respects `system_events_show_zero_values`). [#16438](https://github.com/ClickHouse/ClickHouse/pull/16438) ([Azat Khuzhin](https://github.com/azat)). +* Added function `untuple` which is a special function which can introduce new columns to the SELECT list by expanding a named tuple. [#16242](https://github.com/ClickHouse/ClickHouse/pull/16242) ([Nikolai Kochetov](https://github.com/KochetovNicolai), [Amos Bird](https://github.com/amosbird)). +* Now we can provide identifiers via query parameters. And these parameters can be used as table objects or columns. [#16594](https://github.com/ClickHouse/ClickHouse/pull/16594) ([Amos Bird](https://github.com/amosbird)). +* Added big integers (UInt256, Int128, Int256) and UUID data types support for MergeTree BloomFilter index. Big integers is an experimental feature. [#16642](https://github.com/ClickHouse/ClickHouse/pull/16642) ([Maksim Kita](https://github.com/kitaisreal)). +* Add `farmFingerprint64` function (non-cryptographic string hashing). [#16570](https://github.com/ClickHouse/ClickHouse/pull/16570) ([Jacob Hayes](https://github.com/JacobHayes)). +* Add `log_queries_min_query_duration_ms`, only queries slower then the value of this setting will go to `query_log`/`query_thread_log` (i.e. something like `slow_query_log` in mysql). [#16529](https://github.com/ClickHouse/ClickHouse/pull/16529) ([Azat Khuzhin](https://github.com/azat)). +* Ability to create a docker image on the top of `Alpine`. Uses precompiled binary and glibc components from ubuntu 20.04. [#16479](https://github.com/ClickHouse/ClickHouse/pull/16479) ([filimonov](https://github.com/filimonov)). +* Added `toUUIDOrNull`, `toUUIDOrZero` cast functions. [#16337](https://github.com/ClickHouse/ClickHouse/pull/16337) ([Maksim Kita](https://github.com/kitaisreal)). +* Add `max_concurrent_queries_for_all_users` setting, see [#6636](https://github.com/ClickHouse/ClickHouse/issues/6636) for use cases. [#16154](https://github.com/ClickHouse/ClickHouse/pull/16154) ([nvartolomei](https://github.com/nvartolomei)). +* Add a new option `print_query_id` to clickhouse-client. It helps generate arbitrary strings with the current query id generated by the client. Also print query id in clickhouse-client by default. [#15809](https://github.com/ClickHouse/ClickHouse/pull/15809) ([Amos Bird](https://github.com/amosbird)). +* Add `tid` and `logTrace` functions. This closes [#9434](https://github.com/ClickHouse/ClickHouse/issues/9434). [#15803](https://github.com/ClickHouse/ClickHouse/pull/15803) ([flynn](https://github.com/ucasFL)). +* Add function `formatReadableTimeDelta` that format time delta to human readable string ... [#15497](https://github.com/ClickHouse/ClickHouse/pull/15497) ([Filipe Caixeta](https://github.com/filipecaixeta)). +* Added `disable_merges` option for volumes in multi-disk configuration. [#13956](https://github.com/ClickHouse/ClickHouse/pull/13956) ([Vladimir Chebotarev](https://github.com/excitoon)). + +#### Experimental Feature + +* New functions `encrypt`, `aes_encrypt_mysql`, `decrypt`, `aes_decrypt_mysql`. These functions are working slowly, so we consider it as an experimental feature. [#11844](https://github.com/ClickHouse/ClickHouse/pull/11844) ([Vasily Nemkov](https://github.com/Enmk)). + +#### Bug Fix + +* Mask password in data_path in the `system.distribution_queue`. [#16727](https://github.com/ClickHouse/ClickHouse/pull/16727) ([Azat Khuzhin](https://github.com/azat)). +* Fix `IN` operator over several columns and tuples with enabled `transform_null_in` setting. Fixes [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). +* The setting `max_parallel_replicas` worked incorrectly if the queried table has no sampling. This fixes [#5733](https://github.com/ClickHouse/ClickHouse/issues/5733). [#16675](https://github.com/ClickHouse/ClickHouse/pull/16675) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix optimize_read_in_order/optimize_aggregation_in_order with max_threads > 0 and expression in ORDER BY. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). +* Calculation of `DEFAULT` expressions was involving possible name collisions (that was very unlikely to encounter). This fixes [#9359](https://github.com/ClickHouse/ClickHouse/issues/9359). [#16612](https://github.com/ClickHouse/ClickHouse/pull/16612) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `query_thread_log.query_duration_ms` unit. [#16563](https://github.com/ClickHouse/ClickHouse/pull/16563) ([Azat Khuzhin](https://github.com/azat)). +* Fix a bug when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine. `MaterializeMySQL` is an experimental feature. [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). +* Specifically crafted argument of `round` function with `Decimal` was leading to integer division by zero. This fixes [#13338](https://github.com/ClickHouse/ClickHouse/issues/13338). [#16451](https://github.com/ClickHouse/ClickHouse/pull/16451) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). +* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the inconsistent behaviour when a part of return data could be dropped because the set for its filtration wasn't created. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix dictGet in sharding_key (and similar places, i.e. when the function context is stored permanently). [#16205](https://github.com/ClickHouse/ClickHouse/pull/16205) ([Azat Khuzhin](https://github.com/azat)). +* Fix the exception thrown in `clickhouse-local` when trying to execute `OPTIMIZE` command. Fixes [#16076](https://github.com/ClickHouse/ClickHouse/issues/16076). [#16192](https://github.com/ClickHouse/ClickHouse/pull/16192) ([filimonov](https://github.com/filimonov)). +* Fixes [#15780](https://github.com/ClickHouse/ClickHouse/issues/15780) regression, e.g. `indexOf([1, 2, 3], toLowCardinality(1))` now is prohibited but it should not be. [#16038](https://github.com/ClickHouse/ClickHouse/pull/16038) ([Mike](https://github.com/myrrc)). +* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). +* Now exception will be thrown when `ALTER MODIFY COLUMN ... DEFAULT ...` has incompatible default with column type. Fixes [#15854](https://github.com/ClickHouse/ClickHouse/issues/15854). [#15858](https://github.com/ClickHouse/ClickHouse/pull/15858) ([alesapin](https://github.com/alesapin)). +* Fixed IPv4CIDRToRange/IPv6CIDRToRange functions to accept const IP-column values. [#15856](https://github.com/ClickHouse/ClickHouse/pull/15856) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). + +#### Improvement + +* Treat `INTERVAL '1 hour'` as equivalent to `INTERVAL 1 HOUR`, to be compatible with Postgres and similar. This fixes [#15637](https://github.com/ClickHouse/ClickHouse/issues/15637). [#15978](https://github.com/ClickHouse/ClickHouse/pull/15978) ([flynn](https://github.com/ucasFL)). +* Enable parsing enum values by their numeric ids for CSV, TSV and JSON input formats. [#15685](https://github.com/ClickHouse/ClickHouse/pull/15685) ([vivarum](https://github.com/vivarum)). +* Better read task scheduling for JBOD architecture and `MergeTree` storage. New setting `read_backoff_min_concurrency` which serves as the lower limit to the number of reading threads. [#16423](https://github.com/ClickHouse/ClickHouse/pull/16423) ([Amos Bird](https://github.com/amosbird)). +* Add missing support for `LowCardinality` in `Avro` format. [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). +* Workaround for use `S3` with nginx server as proxy. Nginx currenty does not accept urls with empty path like `http://domain.com?delete`, but vanilla aws-sdk-cpp produces this kind of urls. This commit uses patched aws-sdk-cpp version, which makes urls with "/" as path in this cases, like http://domain.com/?delete. [#16814](https://github.com/ClickHouse/ClickHouse/pull/16814) ([ianton-ru](https://github.com/ianton-ru)). +* Better diagnostics on parse errors in input data. Provide row number on `Cannot read all data` errors. [#16644](https://github.com/ClickHouse/ClickHouse/pull/16644) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make the behaviour of `minMap` and `maxMap` more desireable. It will not skip zero values in the result. Fixes [#16087](https://github.com/ClickHouse/ClickHouse/issues/16087). [#16631](https://github.com/ClickHouse/ClickHouse/pull/16631) ([Ildus Kurbangaliev](https://github.com/ildus)). +* Better update of ZooKeeper configuration in runtime. [#16630](https://github.com/ClickHouse/ClickHouse/pull/16630) ([sundyli](https://github.com/sundy-li)). +* Apply SETTINGS clause as early as possible. It allows to modify more settings in the query. This closes [#3178](https://github.com/ClickHouse/ClickHouse/issues/3178). [#16619](https://github.com/ClickHouse/ClickHouse/pull/16619) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now `event_time_microseconds` field stores in Decimal64, not UInt64. [#16617](https://github.com/ClickHouse/ClickHouse/pull/16617) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Now paratmeterized functions can be used in `APPLY` column transformer. [#16589](https://github.com/ClickHouse/ClickHouse/pull/16589) ([Amos Bird](https://github.com/amosbird)). +* Improve scheduling of background task which removes data of dropped tables in `Atomic` databases. `Atomic` databases do not create broken symlink to table data directory if table actually has no data directory. [#16584](https://github.com/ClickHouse/ClickHouse/pull/16584) ([tavplubix](https://github.com/tavplubix)). +* Subqueries in `WITH` section (CTE) can reference previous subqueries in `WITH` section by their name. [#16575](https://github.com/ClickHouse/ClickHouse/pull/16575) ([Amos Bird](https://github.com/amosbird)). +* Add current_database into `system.query_thread_log`. [#16558](https://github.com/ClickHouse/ClickHouse/pull/16558) ([Azat Khuzhin](https://github.com/azat)). +* Allow to fetch parts that are already committed or outdated in the current instance into the detached directory. It's useful when migrating tables from another cluster and having N to 1 shards mapping. It's also consistent with the current fetchPartition implementation. [#16538](https://github.com/ClickHouse/ClickHouse/pull/16538) ([Amos Bird](https://github.com/amosbird)). +* Multiple improvements for `RabbitMQ`: Fixed bug for [#16263](https://github.com/ClickHouse/ClickHouse/issues/16263). Also minimized event loop lifetime. Added more efficient queues setup. [#16426](https://github.com/ClickHouse/ClickHouse/pull/16426) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix debug assertion in `quantileDeterministic` function. In previous version it may also transfer up to two times more data over the network. Although no bug existed. This fixes [#15683](https://github.com/ClickHouse/ClickHouse/issues/15683). [#16410](https://github.com/ClickHouse/ClickHouse/pull/16410) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add `TablesToDropQueueSize` metric. It's equal to number of dropped tables, that are waiting for background data removal. [#16364](https://github.com/ClickHouse/ClickHouse/pull/16364) ([tavplubix](https://github.com/tavplubix)). +* Better diagnostics when client has dropped connection. In previous versions, `Attempt to read after EOF` and `Broken pipe` exceptions were logged in server. In new version, it's information message `Client has dropped the connection, cancel the query.`. [#16329](https://github.com/ClickHouse/ClickHouse/pull/16329) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add total_rows/total_bytes (from system.tables) support for Set/Join table engines. [#16306](https://github.com/ClickHouse/ClickHouse/pull/16306) ([Azat Khuzhin](https://github.com/azat)). +* Now it's possible to specify `PRIMARY KEY` without `ORDER BY` for MergeTree table engines family. Closes [#15591](https://github.com/ClickHouse/ClickHouse/issues/15591). [#16284](https://github.com/ClickHouse/ClickHouse/pull/16284) ([alesapin](https://github.com/alesapin)). +* If there is no tmp folder in the system (chroot, misconfigutation etc) `clickhouse-local` will create temporary subfolder in the current directory. [#16280](https://github.com/ClickHouse/ClickHouse/pull/16280) ([filimonov](https://github.com/filimonov)). +* Add support for nested data types (like named tuple) as sub-types. Fixes [#15587](https://github.com/ClickHouse/ClickHouse/issues/15587). [#16262](https://github.com/ClickHouse/ClickHouse/pull/16262) ([Ivan](https://github.com/abyss7)). +* Support for `database_atomic_wait_for_drop_and_detach_synchronously`/`NO DELAY`/`SYNC` for `DROP DATABASE`. [#16127](https://github.com/ClickHouse/ClickHouse/pull/16127) ([Azat Khuzhin](https://github.com/azat)). +* Add `allow_nondeterministic_optimize_skip_unused_shards` (to allow non deterministic like `rand()` or `dictGet()` in sharding key). [#16105](https://github.com/ClickHouse/ClickHouse/pull/16105) ([Azat Khuzhin](https://github.com/azat)). +* Fix `memory_profiler_step`/`max_untracked_memory` for queries via HTTP (test included). Fix the issue that adjusting this value globally in xml config does not help either, since those settings are not applied anyway, only default (4MB) value is [used](https://github.com/ClickHouse/ClickHouse/blob/17731245336d8c84f75e4c0894c5797ed7732190/src/Common/ThreadStatus.h#L104). Fix `query_id` for the most root ThreadStatus of the http query (by initializing QueryScope after reading query_id). [#16101](https://github.com/ClickHouse/ClickHouse/pull/16101) ([Azat Khuzhin](https://github.com/azat)). +* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). +* Fix rare issue when `clickhouse-client` may abort on exit due to loading of suggestions. This fixes [#16035](https://github.com/ClickHouse/ClickHouse/issues/16035). [#16047](https://github.com/ClickHouse/ClickHouse/pull/16047) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add support of `cache` layout for `Redis` dictionaries with complex key. [#15985](https://github.com/ClickHouse/ClickHouse/pull/15985) ([Anton Popov](https://github.com/CurtizJ)). +* Fix query hang (endless loop) in case of misconfiguration (`connections_with_failover_max_tries` set to 0). [#15876](https://github.com/ClickHouse/ClickHouse/pull/15876) ([Azat Khuzhin](https://github.com/azat)). +* Change level of some log messages from information to debug, so information messages will not appear for every query. This closes [#5293](https://github.com/ClickHouse/ClickHouse/issues/5293). [#15816](https://github.com/ClickHouse/ClickHouse/pull/15816) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Remove `MemoryTrackingInBackground*` metrics to avoid potentially misleading results. This fixes [#15684](https://github.com/ClickHouse/ClickHouse/issues/15684). [#15813](https://github.com/ClickHouse/ClickHouse/pull/15813) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add reconnects to `zookeeper-dump-tree` tool. [#15711](https://github.com/ClickHouse/ClickHouse/pull/15711) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow explicitly specify columns list in `CREATE TABLE table AS table_function(...)` query. Fixes [#9249](https://github.com/ClickHouse/ClickHouse/issues/9249) Fixes [#14214](https://github.com/ClickHouse/ClickHouse/issues/14214). [#14295](https://github.com/ClickHouse/ClickHouse/pull/14295) ([tavplubix](https://github.com/tavplubix)). + +#### Performance Improvement + +* Do not merge parts across partitions in SELECT FINAL. [#15938](https://github.com/ClickHouse/ClickHouse/pull/15938) ([Kruglov Pavel](https://github.com/Avogar)). +* Improve performance of `-OrNull` and `-OrDefault` aggregate functions. [#16661](https://github.com/ClickHouse/ClickHouse/pull/16661) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improve performance of `quantileMerge`. In previous versions it was obnoxiously slow. This closes [#1463](https://github.com/ClickHouse/ClickHouse/issues/1463). [#16643](https://github.com/ClickHouse/ClickHouse/pull/16643) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improve performance of logical functions a little. [#16347](https://github.com/ClickHouse/ClickHouse/pull/16347) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improved performance of merges assignment in MergeTree table engines. Shouldn't be visible for the user. [#16191](https://github.com/ClickHouse/ClickHouse/pull/16191) ([alesapin](https://github.com/alesapin)). +* Speedup hashed/sparse_hashed dictionary loading by preallocating the hash table. [#15454](https://github.com/ClickHouse/ClickHouse/pull/15454) ([Azat Khuzhin](https://github.com/azat)). +* Now trivial count optimization becomes slightly non-trivial. Predicates that contain exact partition expr can be optimized too. This also fixes [#11092](https://github.com/ClickHouse/ClickHouse/issues/11092) which returns wrong count when `max_parallel_replicas > 1`. [#15074](https://github.com/ClickHouse/ClickHouse/pull/15074) ([Amos Bird](https://github.com/amosbird)). + +#### Build/Testing/Packaging Improvement + +* Add flaky check for stateless tests. It will detect potentially flaky functional tests in advance, before they are merged. [#16238](https://github.com/ClickHouse/ClickHouse/pull/16238) ([alesapin](https://github.com/alesapin)). +* Use proper version for `croaring` instead of amalgamation. [#16285](https://github.com/ClickHouse/ClickHouse/pull/16285) ([sundyli](https://github.com/sundy-li)). +* Improve generation of build files for `ya.make` build system (Arcadia). [#16700](https://github.com/ClickHouse/ClickHouse/pull/16700) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add MySQL BinLog file check tool for `MaterializeMySQL` database engine. `MaterializeMySQL` is an experimental feature. [#16223](https://github.com/ClickHouse/ClickHouse/pull/16223) ([Winter Zhang](https://github.com/zhang2014)). +* Check for executable bit on non-executable files. People often accidentially commit executable files from Windows. [#15843](https://github.com/ClickHouse/ClickHouse/pull/15843) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Check for `#pragma once` in headers. [#15818](https://github.com/ClickHouse/ClickHouse/pull/15818) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix illegal code style `&vector[idx]` in libhdfs3. This fixes libcxx debug build. See also https://github.com/ClickHouse-Extras/libhdfs3/pull/8 . [#15815](https://github.com/ClickHouse/ClickHouse/pull/15815) ([Amos Bird](https://github.com/amosbird)). +* Fix build of one miscellaneous example tool on Mac OS. Note that we don't build examples on Mac OS in our CI (we build only ClickHouse binary), so there is zero chance it will not break again. This fixes [#15804](https://github.com/ClickHouse/ClickHouse/issues/15804). [#15808](https://github.com/ClickHouse/ClickHouse/pull/15808) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Simplify Sys/V init script. [#14135](https://github.com/ClickHouse/ClickHouse/pull/14135) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added `boost::program_options` to `db_generator` in order to increase its usability. This closes [#15940](https://github.com/ClickHouse/ClickHouse/issues/15940). [#15973](https://github.com/ClickHouse/ClickHouse/pull/15973) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + + ## ClickHouse release 20.10 ### ClickHouse release v20.10.3.30, 2020-10-28 From 0b9f0db62074e92aa0c7fd7341ebe204f6403fd5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 12 Nov 2020 01:58:49 +0300 Subject: [PATCH 91/94] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e552fe06962..4474675e9ee 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -62,7 +62,7 @@ * Enable parsing enum values by their numeric ids for CSV, TSV and JSON input formats. [#15685](https://github.com/ClickHouse/ClickHouse/pull/15685) ([vivarum](https://github.com/vivarum)). * Better read task scheduling for JBOD architecture and `MergeTree` storage. New setting `read_backoff_min_concurrency` which serves as the lower limit to the number of reading threads. [#16423](https://github.com/ClickHouse/ClickHouse/pull/16423) ([Amos Bird](https://github.com/amosbird)). * Add missing support for `LowCardinality` in `Avro` format. [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). -* Workaround for use `S3` with nginx server as proxy. Nginx currenty does not accept urls with empty path like `http://domain.com?delete`, but vanilla aws-sdk-cpp produces this kind of urls. This commit uses patched aws-sdk-cpp version, which makes urls with "/" as path in this cases, like http://domain.com/?delete. [#16814](https://github.com/ClickHouse/ClickHouse/pull/16814) ([ianton-ru](https://github.com/ianton-ru)). +* Workaround for use `S3` with nginx server as proxy. Nginx currenty does not accept urls with empty path like `http://domain.com?delete`, but vanilla aws-sdk-cpp produces this kind of urls. This commit uses patched aws-sdk-cpp version, which makes urls with "/" as path in this cases, like `http://domain.com/?delete`. [#16814](https://github.com/ClickHouse/ClickHouse/pull/16814) ([ianton-ru](https://github.com/ianton-ru)). * Better diagnostics on parse errors in input data. Provide row number on `Cannot read all data` errors. [#16644](https://github.com/ClickHouse/ClickHouse/pull/16644) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Make the behaviour of `minMap` and `maxMap` more desireable. It will not skip zero values in the result. Fixes [#16087](https://github.com/ClickHouse/ClickHouse/issues/16087). [#16631](https://github.com/ClickHouse/ClickHouse/pull/16631) ([Ildus Kurbangaliev](https://github.com/ildus)). * Better update of ZooKeeper configuration in runtime. [#16630](https://github.com/ClickHouse/ClickHouse/pull/16630) ([sundyli](https://github.com/sundy-li)). From fe5800a27ec70b3ee438eb07ab6d03bb728fce0d Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Thu, 12 Nov 2020 02:12:51 +0300 Subject: [PATCH 92/94] remove commented code --- src/IO/LZMADeflatingWriteBuffer.cpp | 3 --- src/IO/LZMADeflatingWriteBuffer.h | 1 - src/IO/LZMAInflatingReadBuffer.cpp | 2 -- src/IO/LZMAInflatingReadBuffer.h | 1 - 4 files changed, 7 deletions(-) diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp index 378f7c4ea1b..f309aaa870e 100644 --- a/src/IO/LZMADeflatingWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -12,9 +12,6 @@ LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( std::unique_ptr out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), out(std::move(out_)) { - // FL2_createCStreamMt(number of threads, flag of two dictionaries usage) - // lstr = FL2_createCStreamMt(2, 0); - // size_t res = FL2_initCStream(lstr, compression_level); lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; diff --git a/src/IO/LZMADeflatingWriteBuffer.h b/src/IO/LZMADeflatingWriteBuffer.h index bd733bfa49b..085913f024d 100644 --- a/src/IO/LZMADeflatingWriteBuffer.h +++ b/src/IO/LZMADeflatingWriteBuffer.h @@ -27,7 +27,6 @@ private: std::unique_ptr out; lzma_stream lstr; - // FL2_CStream * lstr; bool finished = false; }; } diff --git a/src/IO/LZMAInflatingReadBuffer.cpp b/src/IO/LZMAInflatingReadBuffer.cpp index b8aa1e703cb..30359987390 100644 --- a/src/IO/LZMAInflatingReadBuffer.cpp +++ b/src/IO/LZMAInflatingReadBuffer.cpp @@ -55,10 +55,8 @@ bool LZMAInflatingReadBuffer::nextImpl() lstr.next_out = reinterpret_cast(internal_buffer.begin()); lstr.avail_out = internal_buffer.size(); - // std::cout << lstr.avail_in << " " << lstr.avail_out << std::endl; lzma_ret ret = lzma_code(&lstr, action); - // std::cout << ret << std::endl; in->position() = in->buffer().end() - lstr.avail_in; working_buffer.resize(internal_buffer.size() - lstr.avail_out); diff --git a/src/IO/LZMAInflatingReadBuffer.h b/src/IO/LZMAInflatingReadBuffer.h index 95c0c8f7681..6994b2f50ee 100644 --- a/src/IO/LZMAInflatingReadBuffer.h +++ b/src/IO/LZMAInflatingReadBuffer.h @@ -27,7 +27,6 @@ private: std::unique_ptr in; lzma_stream lstr; - // FL2_DStream * lstr; bool eof; }; From 20d0c5ca676aba420e50dab2ceba5bd27747fba5 Mon Sep 17 00:00:00 2001 From: Robert Hodges Date: Wed, 11 Nov 2020 14:48:22 -0800 Subject: [PATCH 93/94] Add more known ClickHouse users Added new entries to adopters.md along with links to talks/slides. --- docs/en/introduction/adopters.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index b367a97771a..1cffead788a 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -11,6 +11,7 @@ toc_title: Adopters | Company | Industry | Usecase | Cluster Size | (Un)Compressed Data Size\* | Reference | |------------------------------------------------------------------------------------------------|---------------------------------|-----------------------|------------------------------------------------------------|------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | 2gis | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) | +| Admiral | Martech | Engagement Management | — | — | [Webinar Slides, June 2020](https://altinity.com/presentations/2020/06/16/big-data-in-real-time-how-clickhouse-powers-admirals-visitor-relationships-for-publishers) | | Alibaba Cloud | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) | | Aloha Browser | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) | | Amadeus | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | @@ -29,6 +30,7 @@ toc_title: Adopters | Citadel Securities | Finance | — | — | — | [Contribution, March 2019](https://github.com/ClickHouse/ClickHouse/pull/4774) | | Citymobil | Taxi | Analytics | — | — | [Blog Post in Russian, March 2020](https://habr.com/en/company/citymobil/blog/490660/) | | Cloudflare | CDN | Traffic analysis | 36 servers | — | [Blog post, May 2017](https://blog.cloudflare.com/how-cloudflare-analyzes-1m-dns-queries-per-second/), [Blog post, March 2018](https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/) | +| Comcast | Media | CDN Traffic Analysis | — | — | [ApacheCon 2019 Talk](https://www.youtube.com/watch?v=e9TZ6gFDjNg) | | ContentSquare | Web analytics | Main product | — | — | [Blog post in French, November 2018](http://souslecapot.net/2018/11/21/patrick-chatain-vp-engineering-chez-contentsquare-penser-davantage-amelioration-continue-que-revolution-constante/) | | Corunet | Analytics | Main product | — | — | [Slides in English, April 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup21/predictive_models.pdf) | | CraiditX 氪信 | Finance AI | Analysis | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/udf.pptx) | @@ -64,7 +66,8 @@ toc_title: Adopters | Marilyn | Advertising | Statistics | — | — | [Talk in Russian, June 2017](https://www.youtube.com/watch?v=iXlIgx2khwc) | | Mello | Marketing | Analytics | 1 server | — | [Article, Oct 2020](https://vc.ru/marketing/166180-razrabotka-tipovogo-otcheta-skvoznoy-analitiki) | | MessageBird | Telecommunications | Statistics | — | — | [Slides in English, November 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) | -| MindsDB | Machine Learning | Main Product | — | — | [Official Website](https://www.mindsdb.com/blog/machine-learning-models-as-tables-in-ch) | +| MindsDB | Machine Learning | Main Product | — | — | [Official Website](https://www.mindsdb.com/blog/machine-learning-models-as-tables-in-ch) |x +| MUX | Online Video | Video Analytics | — | — | [Talk in English, August 2019](https://altinity.com/presentations/2019/8/13/how-clickhouse-became-the-default-analytics-database-for-mux/) | | MGID | Ad network | Web-analytics | — | — | [Blog post in Russian, April 2020](http://gs-studio.com/news-about-it/32777----clickhouse---c) | | NOC Project | Network Monitoring | Analytics | Main Product | — | [Official Website](https://getnoc.com/features/big-data/) | | Nuna Inc. | Health Data Analytics | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=170) | From b13189baec7be984fa91d87f3561070332c247b5 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 12 Nov 2020 12:45:08 +0800 Subject: [PATCH 94/94] Try fix fasttest submodule clone --- docker/test/fasttest/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 5b2f324e588..ca5596b91f1 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 contrib/miniselect) +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 contrib/xz) git submodule sync git submodule update --init --recursive "${SUBMODULES_TO_UPDATE[@]}"