mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Implemented some suggestions from clang-tidy (part 1) [#CLICKHOUSE-3301].
This commit is contained in:
parent
1cd09ac12f
commit
e9ae193870
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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.
|
||||
|
@ -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"; }
|
||||
|
@ -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"; }
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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"; }
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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"; }
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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()
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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"; }
|
||||
|
||||
|
@ -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()
|
||||
|
@ -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"; }
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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"; }
|
||||
|
@ -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"; }
|
||||
|
@ -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_);
|
||||
|
@ -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
|
||||
|
@ -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"; }
|
||||
|
@ -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_),
|
||||
|
@ -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"; }
|
||||
|
@ -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>
|
||||
|
@ -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_);
|
||||
|
@ -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())
|
||||
{
|
||||
|
@ -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'};
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
|
@ -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,
|
||||
|
@ -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)
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user