From e9ae1938706c8c6e4aab478c0356b1dcf4d300e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Sep 2017 05:29:47 +0300 Subject: [PATCH] Implemented some suggestions from clang-tidy (part 1) [#CLICKHOUSE-3301]. --- dbms/src/Common/Exception.h | 2 +- dbms/src/Core/ColumnWithTypeAndName.h | 4 ++-- dbms/src/Core/SortDescription.h | 14 ++++++------- .../DataStreams/AggregatingBlockInputStream.h | 4 ++-- .../AsynchronousBlockInputStream.h | 4 ++-- .../DataStreams/BlockExtraInfoInputStream.h | 4 ++-- .../DataStreams/DistinctBlockInputStream.cpp | 4 ++-- .../DataStreams/DistinctBlockInputStream.h | 2 +- .../DistinctSortedBlockInputStream.cpp | 6 +++--- .../DistinctSortedBlockInputStream.h | 4 ++-- .../ExpressionBlockInputStream.cpp | 4 ++-- .../DataStreams/ExpressionBlockInputStream.h | 2 +- .../DataStreams/FilterBlockInputStream.cpp | 8 ++++---- dbms/src/DataStreams/FilterBlockInputStream.h | 4 ++-- dbms/src/DataStreams/ForkBlockInputStreams.h | 2 +- .../src/DataStreams/LimitBlockInputStream.cpp | 4 ++-- dbms/src/DataStreams/LimitBlockInputStream.h | 2 +- .../DataStreams/LimitByBlockInputStream.cpp | 4 ++-- .../src/DataStreams/LimitByBlockInputStream.h | 2 +- .../MaterializingBlockInputStream.cpp | 4 ++-- .../MaterializingBlockInputStream.h | 2 +- .../MergeSortingBlockInputStream.h | 4 ++-- .../MergingAggregatedBlockInputStream.h | 4 ++-- .../NullAndDoCopyBlockInputStream.h | 2 +- .../NullableAdapterBlockInputStream.cpp | 4 ++-- .../NullableAdapterBlockInputStream.h | 2 +- .../ParallelAggregatingBlockInputStream.cpp | 2 +- .../ParallelAggregatingBlockInputStream.h | 2 +- .../src/DataStreams/ParallelInputsProcessor.h | 4 ++-- .../PartialSortingBlockInputStream.h | 2 +- dbms/src/DataStreams/tests/glue_streams.cpp | 2 +- dbms/src/IO/WriteHelpers.h | 8 ++++---- dbms/src/Interpreters/Aggregator.cpp | 4 ++-- dbms/src/Interpreters/Aggregator.h | 4 ++-- .../Interpreters/InterpreterSelectQuery.cpp | 20 +++++++++---------- .../src/Interpreters/InterpreterSelectQuery.h | 4 ++-- dbms/src/Interpreters/SettingsCommon.h | 4 ++-- libs/libmysqlxx/include/mysqlxx/Connection.h | 6 +++--- libs/libmysqlxx/include/mysqlxx/Manip.h | 8 ++++---- 39 files changed, 86 insertions(+), 86 deletions(-) diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index a978ba4a303..756434a73ef 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -116,7 +116,7 @@ typename std::enable_if::value, T>::type exception_cast(std:: { try { - std::rethrow_exception(e); + std::rethrow_exception(std::move(e)); } catch (typename std::remove_pointer::type & concrete) { diff --git a/dbms/src/Core/ColumnWithTypeAndName.h b/dbms/src/Core/ColumnWithTypeAndName.h index 09da42ccf30..edf61430abf 100644 --- a/dbms/src/Core/ColumnWithTypeAndName.h +++ b/dbms/src/Core/ColumnWithTypeAndName.h @@ -19,11 +19,11 @@ struct ColumnWithTypeAndName String name; ColumnWithTypeAndName() {} - ColumnWithTypeAndName(const ColumnPtr & column_, const DataTypePtr & type_, const String name_) + ColumnWithTypeAndName(const ColumnPtr & column_, const DataTypePtr & type_, const String & name_) : column(column_), type(type_), name(name_) {} /// Uses type->createColumn() to create column - ColumnWithTypeAndName(const DataTypePtr & type_, const String name_) + ColumnWithTypeAndName(const DataTypePtr & type_, const String & name_) : column(type_->createColumn()), type(type_), name(name_) {} ColumnWithTypeAndName cloneEmpty() const; diff --git a/dbms/src/Core/SortDescription.h b/dbms/src/Core/SortDescription.h index 70069b4beb9..3c757cc8c21 100644 --- a/dbms/src/Core/SortDescription.h +++ b/dbms/src/Core/SortDescription.h @@ -14,17 +14,17 @@ namespace DB /// Description of the sorting rule by one column. struct SortColumnDescription { - std::string column_name; /// The name of the column. - size_t column_number; /// Column number (used if no name is given). - int direction; /// 1 - ascending, -1 - descending. - int nulls_direction; /// 1 - NULLs and NaNs are greater, -1 - less. - /// To achieve NULLS LAST, set it equal to direction, to achieve NULLS FIRST, set it opposite. - std::shared_ptr collator; /// Collator for locale-specific comparison of strings + std::string column_name; /// The name of the column. + size_t column_number; /// Column number (used if no name is given). + int direction; /// 1 - ascending, -1 - descending. + int nulls_direction; /// 1 - NULLs and NaNs are greater, -1 - less. + /// To achieve NULLS LAST, set it equal to direction, to achieve NULLS FIRST, set it opposite. + std::shared_ptr collator; /// Collator for locale-specific comparison of strings SortColumnDescription(size_t column_number_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr) : column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {} - SortColumnDescription(std::string column_name_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr) + SortColumnDescription(const std::string & column_name_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr) : column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {} /// For IBlockInputStream. diff --git a/dbms/src/DataStreams/AggregatingBlockInputStream.h b/dbms/src/DataStreams/AggregatingBlockInputStream.h index 8bc92c3c3ec..ea93188afba 100644 --- a/dbms/src/DataStreams/AggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/AggregatingBlockInputStream.h @@ -22,10 +22,10 @@ public: * Aggregate functions are searched everywhere in the expression. * Columns corresponding to keys and arguments of aggregate functions must already be computed. */ - AggregatingBlockInputStream(BlockInputStreamPtr input_, const Aggregator::Params & params_, bool final_) + AggregatingBlockInputStream(const BlockInputStreamPtr & input, const Aggregator::Params & params_, bool final_) : params(params_), aggregator(params), final(final_) { - children.push_back(input_); + children.push_back(input); } String getName() const override { return "Aggregating"; } diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index c14cb5bbd90..4ee17c2d30f 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -28,9 +28,9 @@ namespace DB class AsynchronousBlockInputStream : public IProfilingBlockInputStream { public: - AsynchronousBlockInputStream(BlockInputStreamPtr in_) + AsynchronousBlockInputStream(const BlockInputStreamPtr & in) { - children.push_back(in_); + children.push_back(in); } String getName() const override { return "Asynchronous"; } diff --git a/dbms/src/DataStreams/BlockExtraInfoInputStream.h b/dbms/src/DataStreams/BlockExtraInfoInputStream.h index fb3c7e03817..c1a6d77874b 100644 --- a/dbms/src/DataStreams/BlockExtraInfoInputStream.h +++ b/dbms/src/DataStreams/BlockExtraInfoInputStream.h @@ -11,10 +11,10 @@ namespace DB class BlockExtraInfoInputStream : public IProfilingBlockInputStream { public: - BlockExtraInfoInputStream(BlockInputStreamPtr input_, const BlockExtraInfo & block_extra_info_) + BlockExtraInfoInputStream(const BlockInputStreamPtr & input, const BlockExtraInfo & block_extra_info_) : block_extra_info(block_extra_info_) { - children.push_back(input_); + children.push_back(input); } BlockExtraInfo getBlockExtraInfo() const override diff --git a/dbms/src/DataStreams/DistinctBlockInputStream.cpp b/dbms/src/DataStreams/DistinctBlockInputStream.cpp index f27c1ee0f62..d308eb9f17b 100644 --- a/dbms/src/DataStreams/DistinctBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctBlockInputStream.cpp @@ -8,14 +8,14 @@ namespace ErrorCodes extern const int SET_SIZE_LIMIT_EXCEEDED; } -DistinctBlockInputStream::DistinctBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_hint_, Names columns_) +DistinctBlockInputStream::DistinctBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_) : columns_names(columns_) , limit_hint(limit_hint_) , max_rows(limits.max_rows_in_distinct) , max_bytes(limits.max_bytes_in_distinct) , overflow_mode(limits.distinct_overflow_mode) { - children.push_back(input_); + children.push_back(input); } String DistinctBlockInputStream::getID() const diff --git a/dbms/src/DataStreams/DistinctBlockInputStream.h b/dbms/src/DataStreams/DistinctBlockInputStream.h index 41a9e7a2087..7120d3e04a9 100644 --- a/dbms/src/DataStreams/DistinctBlockInputStream.h +++ b/dbms/src/DataStreams/DistinctBlockInputStream.h @@ -18,7 +18,7 @@ class DistinctBlockInputStream : public IProfilingBlockInputStream { public: /// Empty columns_ means all collumns. - DistinctBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_hint_, Names columns_); + DistinctBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_); String getName() const override { return "Distinct"; } diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp index 1dc33a78eab..2c8ab1e1d8e 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp @@ -8,15 +8,15 @@ namespace ErrorCodes extern const int SET_SIZE_LIMIT_EXCEEDED; } -DistinctSortedBlockInputStream::DistinctSortedBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_hint_, Names columns_) - : description( input_->getSortDescription() ) +DistinctSortedBlockInputStream::DistinctSortedBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_) + : description(input->getSortDescription()) , columns_names(columns_) , limit_hint(limit_hint_) , max_rows(limits.max_rows_in_distinct) , max_bytes(limits.max_bytes_in_distinct) , overflow_mode(limits.distinct_overflow_mode) { - children.push_back(input_); + children.push_back(input); } String DistinctSortedBlockInputStream::getID() const diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.h b/dbms/src/DataStreams/DistinctSortedBlockInputStream.h index bf5bbb6c90c..28601f4bb04 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.h +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.h @@ -21,7 +21,7 @@ class DistinctSortedBlockInputStream : public IProfilingBlockInputStream { public: /// Empty columns_ means all collumns. - DistinctSortedBlockInputStream(BlockInputStreamPtr input_, const Limits & limits, size_t limit_hint_, Names columns_); + DistinctSortedBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, Names columns_); String getName() const override { return "DistinctSorted"; } @@ -50,7 +50,7 @@ private: ClearableSetVariants & variants) const; const SortDescription & description; - + struct PreviousBlock { Block block; diff --git a/dbms/src/DataStreams/ExpressionBlockInputStream.cpp b/dbms/src/DataStreams/ExpressionBlockInputStream.cpp index 916de1496b9..6130df31b51 100644 --- a/dbms/src/DataStreams/ExpressionBlockInputStream.cpp +++ b/dbms/src/DataStreams/ExpressionBlockInputStream.cpp @@ -5,10 +5,10 @@ namespace DB { -ExpressionBlockInputStream::ExpressionBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_) +ExpressionBlockInputStream::ExpressionBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_) : expression(expression_) { - children.push_back(input_); + children.push_back(input); } String ExpressionBlockInputStream::getName() const { return "Expression"; } diff --git a/dbms/src/DataStreams/ExpressionBlockInputStream.h b/dbms/src/DataStreams/ExpressionBlockInputStream.h index 6e5b166bad8..c57ea928d54 100644 --- a/dbms/src/DataStreams/ExpressionBlockInputStream.h +++ b/dbms/src/DataStreams/ExpressionBlockInputStream.h @@ -19,7 +19,7 @@ private: using ExpressionActionsPtr = std::shared_ptr; public: - ExpressionBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_); + ExpressionBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_); String getName() const override; String getID() const override; diff --git a/dbms/src/DataStreams/FilterBlockInputStream.cpp b/dbms/src/DataStreams/FilterBlockInputStream.cpp index dcf619bcfd7..be0e9182c08 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.cpp +++ b/dbms/src/DataStreams/FilterBlockInputStream.cpp @@ -17,16 +17,16 @@ namespace ErrorCodes } -FilterBlockInputStream::FilterBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_, ssize_t filter_column_) +FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, ssize_t filter_column_) : expression(expression_), filter_column(filter_column_) { - children.push_back(input_); + children.push_back(input); } -FilterBlockInputStream::FilterBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_, const String & filter_column_name_) +FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, const String & filter_column_name_) : expression(expression_), filter_column(-1), filter_column_name(filter_column_name_) { - children.push_back(input_); + children.push_back(input); } diff --git a/dbms/src/DataStreams/FilterBlockInputStream.h b/dbms/src/DataStreams/FilterBlockInputStream.h index 3f0cda420c6..95ade939256 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.h +++ b/dbms/src/DataStreams/FilterBlockInputStream.h @@ -20,8 +20,8 @@ private: public: /// filter_column_ - the number of the column with filter conditions. - FilterBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_, ssize_t filter_column_); - FilterBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_, const String & filter_column_name_); + FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, ssize_t filter_column_); + FilterBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_, const String & filter_column_name_); String getName() const override; String getID() const override; diff --git a/dbms/src/DataStreams/ForkBlockInputStreams.h b/dbms/src/DataStreams/ForkBlockInputStreams.h index e10e0d3f8fd..504cfc5bdbe 100644 --- a/dbms/src/DataStreams/ForkBlockInputStreams.h +++ b/dbms/src/DataStreams/ForkBlockInputStreams.h @@ -18,7 +18,7 @@ namespace DB class ForkBlockInputStreams : private boost::noncopyable { public: - ForkBlockInputStreams(BlockInputStreamPtr source_) : source(source_) {} + ForkBlockInputStreams(const BlockInputStreamPtr & source_) : source(source_) {} /// Create a source. Call the function as many times as many forked sources you need. BlockInputStreamPtr createInput() diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index 56edf6d1e98..7939d1f73ca 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -6,10 +6,10 @@ namespace DB { -LimitBlockInputStream::LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_, bool always_read_till_end_) +LimitBlockInputStream::LimitBlockInputStream(const BlockInputStreamPtr & input, size_t limit_, size_t offset_, bool always_read_till_end_) : limit(limit_), offset(offset_), always_read_till_end(always_read_till_end_) { - children.push_back(input_); + children.push_back(input); } diff --git a/dbms/src/DataStreams/LimitBlockInputStream.h b/dbms/src/DataStreams/LimitBlockInputStream.h index 1588441ee91..23bd5739aaf 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.h +++ b/dbms/src/DataStreams/LimitBlockInputStream.h @@ -17,7 +17,7 @@ public: * If always_read_till_end = true - reads all the data to the end, but ignores them. This is necessary in rare cases: * when otherwise, due to the cancellation of the request, we would not have received the data for GROUP BY WITH TOTALS from the remote server. */ - LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_, bool always_read_till_end_ = false); + LimitBlockInputStream(const BlockInputStreamPtr & input, size_t limit_, size_t offset_, bool always_read_till_end_ = false); String getName() const override { return "Limit"; } diff --git a/dbms/src/DataStreams/LimitByBlockInputStream.cpp b/dbms/src/DataStreams/LimitByBlockInputStream.cpp index 8e0125a891e..6182dd3413b 100644 --- a/dbms/src/DataStreams/LimitByBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitByBlockInputStream.cpp @@ -3,11 +3,11 @@ namespace DB { -LimitByBlockInputStream::LimitByBlockInputStream(BlockInputStreamPtr input_, size_t group_size_, Names columns_) +LimitByBlockInputStream::LimitByBlockInputStream(const BlockInputStreamPtr & input, size_t group_size_, Names columns_) : columns_names(columns_) , group_size(group_size_) { - children.push_back(input_); + children.push_back(input); } Block LimitByBlockInputStream::readImpl() diff --git a/dbms/src/DataStreams/LimitByBlockInputStream.h b/dbms/src/DataStreams/LimitByBlockInputStream.h index 0460e799ad6..54874d37f1e 100644 --- a/dbms/src/DataStreams/LimitByBlockInputStream.h +++ b/dbms/src/DataStreams/LimitByBlockInputStream.h @@ -18,7 +18,7 @@ namespace DB class LimitByBlockInputStream : public IProfilingBlockInputStream { public: - LimitByBlockInputStream(BlockInputStreamPtr input_, size_t group_size_, Names columns_); + LimitByBlockInputStream(const BlockInputStreamPtr & input, size_t group_size_, Names columns_); String getName() const override { return "LimitBy"; } diff --git a/dbms/src/DataStreams/MaterializingBlockInputStream.cpp b/dbms/src/DataStreams/MaterializingBlockInputStream.cpp index fefdfb3be26..640eb87b103 100644 --- a/dbms/src/DataStreams/MaterializingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MaterializingBlockInputStream.cpp @@ -5,9 +5,9 @@ namespace DB { -MaterializingBlockInputStream::MaterializingBlockInputStream(BlockInputStreamPtr input_) +MaterializingBlockInputStream::MaterializingBlockInputStream(const BlockInputStreamPtr & input) { - children.push_back(input_); + children.push_back(input); } String MaterializingBlockInputStream::getName() const diff --git a/dbms/src/DataStreams/MaterializingBlockInputStream.h b/dbms/src/DataStreams/MaterializingBlockInputStream.h index 654249e309a..feeca05298e 100644 --- a/dbms/src/DataStreams/MaterializingBlockInputStream.h +++ b/dbms/src/DataStreams/MaterializingBlockInputStream.h @@ -10,7 +10,7 @@ namespace DB class MaterializingBlockInputStream : public IProfilingBlockInputStream { public: - MaterializingBlockInputStream(BlockInputStreamPtr input_); + MaterializingBlockInputStream(const BlockInputStreamPtr & input); String getName() const override; String getID() const override; diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index ce59bb184b0..4a97d214a79 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -69,13 +69,13 @@ class MergeSortingBlockInputStream : public IProfilingBlockInputStream { public: /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. - MergeSortingBlockInputStream(BlockInputStreamPtr input_, SortDescription & description_, + MergeSortingBlockInputStream(const BlockInputStreamPtr & input, SortDescription & description_, size_t max_merged_block_size_, size_t limit_, size_t max_bytes_before_external_sort_, const std::string & tmp_path_) : description(description_), max_merged_block_size(max_merged_block_size_), limit(limit_), max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_path(tmp_path_) { - children.push_back(input_); + children.push_back(input); } String getName() const override { return "MergeSorting"; } diff --git a/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h b/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h index 2346f7bd221..f133524d0f1 100644 --- a/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h +++ b/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h @@ -14,10 +14,10 @@ namespace DB class MergingAggregatedBlockInputStream : public IProfilingBlockInputStream { public: - MergingAggregatedBlockInputStream(BlockInputStreamPtr input_, const Aggregator::Params & params, bool final_, size_t max_threads_) + MergingAggregatedBlockInputStream(const BlockInputStreamPtr & input, const Aggregator::Params & params, bool final_, size_t max_threads_) : aggregator(params), final(final_), max_threads(max_threads_) { - children.push_back(input_); + children.push_back(input); } String getName() const override { return "MergingAggregated"; } diff --git a/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h b/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h index 3aa39e2da23..cd7823f236d 100644 --- a/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h +++ b/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h @@ -20,7 +20,7 @@ using BlockOutputStreamPtr = std::shared_ptr; class NullAndDoCopyBlockInputStream : public IProfilingBlockInputStream { public: - NullAndDoCopyBlockInputStream(BlockInputStreamPtr input_, BlockOutputStreamPtr output_) + NullAndDoCopyBlockInputStream(const BlockInputStreamPtr & input_, BlockOutputStreamPtr output_) : input(input_), output(output_) { children.push_back(input_); diff --git a/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp b/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp index c7b87d4b86d..49a349ec980 100644 --- a/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp +++ b/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp @@ -17,11 +17,11 @@ extern const int TYPE_MISMATCH; } NullableAdapterBlockInputStream::NullableAdapterBlockInputStream( - BlockInputStreamPtr input_, + const BlockInputStreamPtr & input, const Block & in_sample_, const Block & out_sample_) { buildActions(in_sample_, out_sample_); - children.push_back(input_); + children.push_back(input); } String NullableAdapterBlockInputStream::getID() const diff --git a/dbms/src/DataStreams/NullableAdapterBlockInputStream.h b/dbms/src/DataStreams/NullableAdapterBlockInputStream.h index e58920cd147..917733cc6d4 100644 --- a/dbms/src/DataStreams/NullableAdapterBlockInputStream.h +++ b/dbms/src/DataStreams/NullableAdapterBlockInputStream.h @@ -18,7 +18,7 @@ namespace DB class NullableAdapterBlockInputStream : public IProfilingBlockInputStream { public: - NullableAdapterBlockInputStream(BlockInputStreamPtr input_, const Block & in_sample_, + NullableAdapterBlockInputStream(const BlockInputStreamPtr & input, const Block & in_sample_, const Block & out_sample_); String getName() const override { return "NullableAdapterBlockInputStream"; } diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index 40c71d244fc..92bfe5e1f9e 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -16,7 +16,7 @@ namespace DB ParallelAggregatingBlockInputStream::ParallelAggregatingBlockInputStream( - BlockInputStreams inputs, BlockInputStreamPtr additional_input_at_end, + const BlockInputStreams & inputs, const BlockInputStreamPtr & additional_input_at_end, const Aggregator::Params & params_, bool final_, size_t max_threads_, size_t temporary_data_merge_threads_) : params(params_), aggregator(params), final(final_), max_threads(std::min(inputs.size(), max_threads_)), temporary_data_merge_threads(temporary_data_merge_threads_), diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h index 3bc08e75ea3..f76ef4e13b8 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h @@ -22,7 +22,7 @@ public: /** Columns from key_names and arguments of aggregate functions must already be computed. */ ParallelAggregatingBlockInputStream( - BlockInputStreams inputs, BlockInputStreamPtr additional_input_at_end, + const BlockInputStreams & inputs, const BlockInputStreamPtr & additional_input_at_end, const Aggregator::Params & params_, bool final_, size_t max_threads_, size_t temporary_data_merge_threads_); String getName() const override { return "ParallelAggregating"; } diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index cdf17e1ab4f..c00c9f26ffa 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -76,7 +76,7 @@ public: * - where you must first make JOIN in parallel, while noting which keys are not found, * and only after the completion of this work, create blocks of keys that are not found. */ - ParallelInputsProcessor(BlockInputStreams inputs_, BlockInputStreamPtr additional_input_at_end_, size_t max_threads_, Handler & handler_) + ParallelInputsProcessor(const BlockInputStreams & inputs_, const BlockInputStreamPtr & additional_input_at_end_, size_t max_threads_, Handler & handler_) : inputs(inputs_), additional_input_at_end(additional_input_at_end_), max_threads(std::min(inputs_.size(), max_threads_)), handler(handler_) { for (size_t i = 0; i < inputs_.size(); ++i) @@ -155,7 +155,7 @@ private: size_t i; /// The source number (for debugging). InputData() {} - InputData(BlockInputStreamPtr & in_, size_t i_) : in(in_), i(i_) {} + InputData(const BlockInputStreamPtr & in_, size_t i_) : in(in_), i(i_) {} }; template diff --git a/dbms/src/DataStreams/PartialSortingBlockInputStream.h b/dbms/src/DataStreams/PartialSortingBlockInputStream.h index 924fdb0e2bf..f46fa8e5ca5 100644 --- a/dbms/src/DataStreams/PartialSortingBlockInputStream.h +++ b/dbms/src/DataStreams/PartialSortingBlockInputStream.h @@ -15,7 +15,7 @@ class PartialSortingBlockInputStream : public IProfilingBlockInputStream { public: /// limit - if not 0, then you can sort each block not completely, but only `limit` first rows by order. - PartialSortingBlockInputStream(BlockInputStreamPtr input_, SortDescription & description_, size_t limit_ = 0) + PartialSortingBlockInputStream(const BlockInputStreamPtr & input_, SortDescription & description_, size_t limit_ = 0) : description(description_), limit(limit_) { children.push_back(input_); diff --git a/dbms/src/DataStreams/tests/glue_streams.cpp b/dbms/src/DataStreams/tests/glue_streams.cpp index 78205b9c0ce..8982f2c0ef6 100644 --- a/dbms/src/DataStreams/tests/glue_streams.cpp +++ b/dbms/src/DataStreams/tests/glue_streams.cpp @@ -15,7 +15,7 @@ using namespace DB; -void inputThread(BlockInputStreamPtr in, BlockOutputStreamPtr out, WriteBuffer & wb, std::mutex & mutex) +void inputThread(const BlockInputStreamPtr & in, BlockOutputStreamPtr out, WriteBuffer & wb, std::mutex & mutex) { while (Block block = in->read()) { diff --git a/dbms/src/IO/WriteHelpers.h b/dbms/src/IO/WriteHelpers.h index 76def04e0a4..6c0cf219bac 100644 --- a/dbms/src/IO/WriteHelpers.h +++ b/dbms/src/IO/WriteHelpers.h @@ -197,7 +197,7 @@ inline void writeJSONString(const char * begin, const char * end, WriteBuffer & break; default: UInt8 c = *it; - if (0x00 <= c && c <= 0x1F) + if (c <= 0x1F) { /// Escaping of ASCII control characters. @@ -207,7 +207,7 @@ inline void writeJSONString(const char * begin, const char * end, WriteBuffer & writeCString("\\u00", buf); writeChar('0' + higher_half, buf); - if (0 <= lower_half && lower_half <= 9) + if (lower_half <= 9) writeChar('0' + lower_half, buf); else writeChar('A' + lower_half - 10, buf); @@ -522,7 +522,7 @@ inline void writeDateText(DayNum_t date, WriteBuffer & buf) buf.write(s, 10); } -inline void writeDateText(LocalDate date, WriteBuffer & buf) +inline void writeDateText(const LocalDate & date, WriteBuffer & buf) { char s[10] = {'0', '0', '0', '0', '-', '0', '0', '-', '0', '0'}; @@ -577,7 +577,7 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTI } template -inline void writeDateTimeText(LocalDateTime datetime, WriteBuffer & buf) +inline void writeDateTimeText(const LocalDateTime & datetime, WriteBuffer & buf) { char s[19] = {'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0', ' ', '0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0'}; diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 8b038f6aa67..be4badb2af0 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -1034,7 +1034,7 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const } -void Aggregator::execute(BlockInputStreamPtr stream, AggregatedDataVariants & result) +void Aggregator::execute(const BlockInputStreamPtr & stream, AggregatedDataVariants & result) { if (isCancelled()) return; @@ -1989,7 +1989,7 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl( } -void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants & result, size_t max_threads) +void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataVariants & result, size_t max_threads) { if (isCancelled()) return; diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 2003759fbbf..ee0b0e6cd87 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -1039,7 +1039,7 @@ public: } /// Aggregate the source. Get the result in the form of one of the data structures. - void execute(BlockInputStreamPtr stream, AggregatedDataVariants & result); + void execute(const BlockInputStreamPtr & stream, AggregatedDataVariants & result); using AggregateColumns = std::vector; using AggregateColumnsData = std::vector; @@ -1067,7 +1067,7 @@ public: /** Merge the stream of partially aggregated blocks into one data structure. * (Pre-aggregate several blocks that represent the result of independent aggregations from remote servers.) */ - void mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants & result, size_t max_threads); + void mergeStream(const BlockInputStreamPtr & stream, AggregatedDataVariants & result, size_t max_threads); /// Merge several partially aggregated blocks into one. /// Precondition: for all blocks block.info.is_overflows flag must be the same. diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index c5518e7de35..da2a9e2e0d4 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -69,7 +69,7 @@ namespace ErrorCodes InterpreterSelectQuery::~InterpreterSelectQuery() = default; -void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & required_column_names) +void InterpreterSelectQuery::init(const BlockInputStreamPtr & input, const Names & required_column_names) { ProfileEvents::increment(ProfileEvents::SelectQuery); @@ -126,7 +126,7 @@ void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & requi } } -void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_) +void InterpreterSelectQuery::basicInit(const BlockInputStreamPtr & input) { auto query_table = query.table(); @@ -171,8 +171,8 @@ void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_) if (!context.tryGetExternalTable(it.first)) context.addExternalTable(it.first, it.second); - if (input_) - streams.push_back(input_); + if (input) + streams.push_back(input); if (is_first_select_inside_union_all) { @@ -200,7 +200,7 @@ void InterpreterSelectQuery::initQueryAnalyzer() } InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const Context & context_, QueryProcessingStage::Enum to_stage_, - size_t subquery_depth_, BlockInputStreamPtr input_) + size_t subquery_depth_, BlockInputStreamPtr input) : query_ptr(query_ptr_) , query(typeid_cast(*query_ptr)) , context(context_) @@ -209,7 +209,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const , is_first_select_inside_union_all(query.isUnionAllHead()) , log(&Logger::get("InterpreterSelectQuery")) { - init(input_); + init(input); } InterpreterSelectQuery::InterpreterSelectQuery(OnlyAnalyzeTag, const ASTPtr & query_ptr_, const Context & context_) @@ -226,14 +226,14 @@ InterpreterSelectQuery::InterpreterSelectQuery(OnlyAnalyzeTag, const ASTPtr & qu InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const Context & context_, const Names & required_column_names_, - QueryProcessingStage::Enum to_stage_, size_t subquery_depth_, BlockInputStreamPtr input_) - : InterpreterSelectQuery(query_ptr_, context_, required_column_names_, {}, to_stage_, subquery_depth_, input_) + QueryProcessingStage::Enum to_stage_, size_t subquery_depth_, BlockInputStreamPtr input) + : InterpreterSelectQuery(query_ptr_, context_, required_column_names_, {}, to_stage_, subquery_depth_, input) { } InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const Context & context_, const Names & required_column_names_, - const NamesAndTypesList & table_column_names_, QueryProcessingStage::Enum to_stage_, size_t subquery_depth_, BlockInputStreamPtr input_) + const NamesAndTypesList & table_column_names_, QueryProcessingStage::Enum to_stage_, size_t subquery_depth_, BlockInputStreamPtr input) : query_ptr(query_ptr_) , query(typeid_cast(*query_ptr)) , context(context_) @@ -243,7 +243,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const , is_first_select_inside_union_all(query.isUnionAllHead()) , log(&Logger::get("InterpreterSelectQuery")) { - init(input_, required_column_names_); + init(input, required_column_names_); } bool InterpreterSelectQuery::hasAsterisk() const diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 83b1722b2cc..a7e8ea1445e 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -93,8 +93,8 @@ private: const ASTPtr & query_ptr_, const Context & context_); - void init(BlockInputStreamPtr input, const Names & required_column_names = Names{}); - void basicInit(BlockInputStreamPtr input); + void init(const BlockInputStreamPtr & input, const Names & required_column_names = Names{}); + void basicInit(const BlockInputStreamPtr & input); void initQueryAnalyzer(); /// Execute one SELECT query from the UNION ALL chain. diff --git a/dbms/src/Interpreters/SettingsCommon.h b/dbms/src/Interpreters/SettingsCommon.h index e3cf90eee11..69fbee54d9e 100644 --- a/dbms/src/Interpreters/SettingsCommon.h +++ b/dbms/src/Interpreters/SettingsCommon.h @@ -219,7 +219,7 @@ struct SettingMilliseconds SettingMilliseconds(UInt64 milliseconds = 0) : value(milliseconds * 1000) {} operator Poco::Timespan() const { return value; } - SettingMilliseconds & operator= (Poco::Timespan x) { set(x); return *this; } + SettingMilliseconds & operator= (const Poco::Timespan & x) { set(x); return *this; } Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); } @@ -228,7 +228,7 @@ struct SettingMilliseconds return DB::toString(totalMilliseconds()); } - void set(Poco::Timespan x) + void set(const Poco::Timespan & x) { value = x; changed = true; diff --git a/libs/libmysqlxx/include/mysqlxx/Connection.h b/libs/libmysqlxx/include/mysqlxx/Connection.h index 7ecb3c6fdc5..c704faa3212 100644 --- a/libs/libmysqlxx/include/mysqlxx/Connection.h +++ b/libs/libmysqlxx/include/mysqlxx/Connection.h @@ -47,7 +47,7 @@ private: * In order to use connection in other thread, you should call MySQL C API function mysql_thread_init() before and * mysql_thread_end() after working with it. */ -class Connection : private boost::noncopyable +class Connection final : private boost::noncopyable { public: /// For delayed initialisation @@ -69,10 +69,10 @@ public: /// All settings will be got from config_name section of configuration. Connection(const std::string & config_name); - virtual ~Connection(); + ~Connection(); /// Provides delayed initialization or reconnection with other settings. - virtual void connect(const char * db, + void connect(const char * db, const char * server, const char * user, const char * password, diff --git a/libs/libmysqlxx/include/mysqlxx/Manip.h b/libs/libmysqlxx/include/mysqlxx/Manip.h index 11fab8264dd..c02d9e10515 100644 --- a/libs/libmysqlxx/include/mysqlxx/Manip.h +++ b/libs/libmysqlxx/include/mysqlxx/Manip.h @@ -50,8 +50,8 @@ struct EscapeManipResult typename std::enable_if::value, std::ostream &>::type operator<< (T value) { return ostr << value; } - std::ostream & operator<< (LocalDate value) { return ostr << value; } - std::ostream & operator<< (LocalDateTime value) { return ostr << value; } + std::ostream & operator<< (const LocalDate & value) { return ostr << value; } + std::ostream & operator<< (const LocalDateTime & value) { return ostr << value; } std::ostream & operator<< (const std::string & value) { @@ -172,8 +172,8 @@ public: typename std::enable_if::value, std::ostream &>::type operator<< (T value) { return ostr << value; } - std::ostream & operator<< (LocalDate value) { return ostr << '\'' << value << '\''; } - std::ostream & operator<< (LocalDateTime value) { return ostr << '\'' << value << '\''; } + std::ostream & operator<< (const LocalDate & value) { return ostr << '\'' << value << '\''; } + std::ostream & operator<< (const LocalDateTime & value) { return ostr << '\'' << value << '\''; } std::ostream & operator<< (const std::string & value) {