Implemented some suggestions from clang-tidy (part 1) [#CLICKHOUSE-3301].

This commit is contained in:
Alexey Milovidov 2017-09-08 05:29:47 +03:00
parent 1cd09ac12f
commit e9ae193870
39 changed files with 86 additions and 86 deletions

View File

@ -116,7 +116,7 @@ typename std::enable_if<std::is_pointer<T>::value, T>::type exception_cast(std::
{
try
{
std::rethrow_exception(e);
std::rethrow_exception(std::move(e));
}
catch (typename std::remove_pointer<T>::type & concrete)
{

View File

@ -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;

View File

@ -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; /// 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; /// Collator for locale-specific comparison of strings
SortColumnDescription(size_t column_number_, int direction_, int nulls_direction_, const std::shared_ptr<Collator> & 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> & collator_ = nullptr)
SortColumnDescription(const std::string & column_name_, int direction_, int nulls_direction_, const std::shared_ptr<Collator> & collator_ = nullptr)
: column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {}
/// For IBlockInputStream.

View File

@ -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"; }

View File

@ -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"; }

View File

@ -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

View File

@ -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

View File

@ -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"; }

View File

@ -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

View File

@ -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;

View File

@ -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"; }

View File

@ -19,7 +19,7 @@ private:
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
public:
ExpressionBlockInputStream(BlockInputStreamPtr input_, ExpressionActionsPtr expression_);
ExpressionBlockInputStream(const BlockInputStreamPtr & input, ExpressionActionsPtr expression_);
String getName() const override;
String getID() const override;

View File

@ -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);
}

View File

@ -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;

View File

@ -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()

View File

@ -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);
}

View File

@ -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"; }

View File

@ -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()

View File

@ -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"; }

View File

@ -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

View File

@ -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;

View File

@ -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"; }

View File

@ -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"; }

View File

@ -20,7 +20,7 @@ using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
class NullAndDoCopyBlockInputStream : public IProfilingBlockInputStream
{
public:
NullAndDoCopyBlockInputStream(BlockInputStreamPtr input_, BlockOutputStreamPtr output_)
NullAndDoCopyBlockInputStream(const BlockInputStreamPtr & input_, BlockOutputStreamPtr output_)
: input(input_), output(output_)
{
children.push_back(input_);

View File

@ -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

View File

@ -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"; }

View File

@ -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_),

View File

@ -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"; }

View File

@ -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 <StreamUnionMode mode2 = mode>

View File

@ -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_);

View File

@ -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())
{

View File

@ -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 <char date_delimeter = '-', char time_delimeter = ':'>
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'};

View File

@ -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;

View File

@ -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<ConstColumnPlainPtrs>;
using AggregateColumnsData = std::vector<ColumnAggregateFunction::Container_t *>;
@ -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.

View File

@ -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<ASTSelectQuery &>(*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<ASTSelectQuery &>(*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

View File

@ -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.

View File

@ -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;

View File

@ -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,

View File

@ -50,8 +50,8 @@ struct EscapeManipResult
typename std::enable_if<std::is_arithmetic<T>::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<std::is_arithmetic<T>::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)
{