mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-11 18:14:03 +00:00
Enabling -Wshadow [#CLICKHOUSE-2]
This commit is contained in:
parent
0d02315778
commit
a3cd1ea8cb
@ -106,7 +106,6 @@ void ColumnAggregateFunction::insertRangeFrom(const IColumn & from, size_t start
|
|||||||
/// Keep shared ownership of aggregation states.
|
/// Keep shared ownership of aggregation states.
|
||||||
src = from_concrete.getPtr();
|
src = from_concrete.getPtr();
|
||||||
|
|
||||||
auto & data = getData();
|
|
||||||
size_t old_size = data.size();
|
size_t old_size = data.size();
|
||||||
data.resize(old_size + length);
|
data.resize(old_size + length);
|
||||||
memcpy(&data[old_size], &from_concrete.getData()[start], length * sizeof(data[0]));
|
memcpy(&data[old_size], &from_concrete.getData()[start], length * sizeof(data[0]));
|
||||||
@ -246,13 +245,13 @@ void ColumnAggregateFunction::insertData(const char * pos, size_t /*length*/)
|
|||||||
getData().push_back(*reinterpret_cast<const AggregateDataPtr *>(pos));
|
getData().push_back(*reinterpret_cast<const AggregateDataPtr *>(pos));
|
||||||
}
|
}
|
||||||
|
|
||||||
void ColumnAggregateFunction::insertFrom(const IColumn & src, size_t n)
|
void ColumnAggregateFunction::insertFrom(const IColumn & from, size_t n)
|
||||||
{
|
{
|
||||||
/// Must create new state of aggregate function and take ownership of it,
|
/// Must create new state of aggregate function and take ownership of it,
|
||||||
/// because ownership of states of aggregate function cannot be shared for individual rows,
|
/// because ownership of states of aggregate function cannot be shared for individual rows,
|
||||||
/// (only as a whole, see comment above).
|
/// (only as a whole, see comment above).
|
||||||
insertDefault();
|
insertDefault();
|
||||||
insertMergeFrom(src, n);
|
insertMergeFrom(from, n);
|
||||||
}
|
}
|
||||||
|
|
||||||
void ColumnAggregateFunction::insertFrom(ConstAggregateDataPtr place)
|
void ColumnAggregateFunction::insertFrom(ConstAggregateDataPtr place)
|
||||||
@ -266,9 +265,9 @@ void ColumnAggregateFunction::insertMergeFrom(ConstAggregateDataPtr place)
|
|||||||
func->merge(getData().back(), place, &createOrGetArena());
|
func->merge(getData().back(), place, &createOrGetArena());
|
||||||
}
|
}
|
||||||
|
|
||||||
void ColumnAggregateFunction::insertMergeFrom(const IColumn & src, size_t n)
|
void ColumnAggregateFunction::insertMergeFrom(const IColumn & from, size_t n)
|
||||||
{
|
{
|
||||||
insertMergeFrom(static_cast<const ColumnAggregateFunction &>(src).getData()[n]);
|
insertMergeFrom(static_cast<const ColumnAggregateFunction &>(from).getData()[n]);
|
||||||
}
|
}
|
||||||
|
|
||||||
Arena & ColumnAggregateFunction::createOrGetArena()
|
Arena & ColumnAggregateFunction::createOrGetArena()
|
||||||
|
@ -123,14 +123,14 @@ public:
|
|||||||
|
|
||||||
void insertData(const char * pos, size_t length) override;
|
void insertData(const char * pos, size_t length) override;
|
||||||
|
|
||||||
void insertFrom(const IColumn & src, size_t n) override;
|
void insertFrom(const IColumn & from, size_t n) override;
|
||||||
|
|
||||||
void insertFrom(ConstAggregateDataPtr place);
|
void insertFrom(ConstAggregateDataPtr place);
|
||||||
|
|
||||||
/// Merge state at last row with specified state in another column.
|
/// Merge state at last row with specified state in another column.
|
||||||
void insertMergeFrom(ConstAggregateDataPtr place);
|
void insertMergeFrom(ConstAggregateDataPtr place);
|
||||||
|
|
||||||
void insertMergeFrom(const IColumn & src, size_t n);
|
void insertMergeFrom(const IColumn & from, size_t n);
|
||||||
|
|
||||||
Arena & createOrGetArena();
|
Arena & createOrGetArena();
|
||||||
|
|
||||||
|
@ -237,11 +237,11 @@ void ColumnArray::insertDefault()
|
|||||||
|
|
||||||
void ColumnArray::popBack(size_t n)
|
void ColumnArray::popBack(size_t n)
|
||||||
{
|
{
|
||||||
auto & offsets = getOffsets();
|
auto & offsets_data = getOffsets();
|
||||||
size_t nested_n = offsets.back() - offsetAt(offsets.size() - n);
|
size_t nested_n = offsets_data.back() - offsetAt(offsets_data.size() - n);
|
||||||
if (nested_n)
|
if (nested_n)
|
||||||
getData().popBack(nested_n);
|
getData().popBack(nested_n);
|
||||||
offsets.resize_assume_reserved(offsets.size() - n);
|
offsets_data.resize_assume_reserved(offsets_data.size() - n);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -914,10 +914,10 @@ Block Aggregator::convertOneBucketToBlock(
|
|||||||
MutableColumns & key_columns,
|
MutableColumns & key_columns,
|
||||||
AggregateColumnsData & aggregate_columns,
|
AggregateColumnsData & aggregate_columns,
|
||||||
MutableColumns & final_aggregate_columns,
|
MutableColumns & final_aggregate_columns,
|
||||||
bool final)
|
bool final_)
|
||||||
{
|
{
|
||||||
convertToBlockImpl(method, method.data.impls[bucket],
|
convertToBlockImpl(method, method.data.impls[bucket],
|
||||||
key_columns, aggregate_columns, final_aggregate_columns, final);
|
key_columns, aggregate_columns, final_aggregate_columns, final_);
|
||||||
});
|
});
|
||||||
|
|
||||||
block.info.bucket_num = bucket;
|
block.info.bucket_num = bucket;
|
||||||
@ -1190,7 +1190,7 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va
|
|||||||
MutableColumns & key_columns,
|
MutableColumns & key_columns,
|
||||||
AggregateColumnsData & aggregate_columns,
|
AggregateColumnsData & aggregate_columns,
|
||||||
MutableColumns & final_aggregate_columns,
|
MutableColumns & final_aggregate_columns,
|
||||||
bool final)
|
bool final_)
|
||||||
{
|
{
|
||||||
if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
|
if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
|
||||||
{
|
{
|
||||||
@ -1198,13 +1198,13 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va
|
|||||||
|
|
||||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||||
{
|
{
|
||||||
if (!final)
|
if (!final_)
|
||||||
aggregate_columns[i]->push_back(data + offsets_of_aggregate_states[i]);
|
aggregate_columns[i]->push_back(data + offsets_of_aggregate_states[i]);
|
||||||
else
|
else
|
||||||
aggregate_functions[i]->insertResultInto(data + offsets_of_aggregate_states[i], *final_aggregate_columns[i]);
|
aggregate_functions[i]->insertResultInto(data + offsets_of_aggregate_states[i], *final_aggregate_columns[i]);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!final)
|
if (!final_)
|
||||||
data = nullptr;
|
data = nullptr;
|
||||||
|
|
||||||
if (params.overflow_row)
|
if (params.overflow_row)
|
||||||
@ -1232,12 +1232,12 @@ Block Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_v
|
|||||||
MutableColumns & key_columns,
|
MutableColumns & key_columns,
|
||||||
AggregateColumnsData & aggregate_columns,
|
AggregateColumnsData & aggregate_columns,
|
||||||
MutableColumns & final_aggregate_columns,
|
MutableColumns & final_aggregate_columns,
|
||||||
bool final)
|
bool final_)
|
||||||
{
|
{
|
||||||
#define M(NAME) \
|
#define M(NAME) \
|
||||||
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
|
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
|
||||||
convertToBlockImpl(*data_variants.NAME, data_variants.NAME->data, \
|
convertToBlockImpl(*data_variants.NAME, data_variants.NAME->data, \
|
||||||
key_columns, aggregate_columns, final_aggregate_columns, final);
|
key_columns, aggregate_columns, final_aggregate_columns, final_);
|
||||||
|
|
||||||
if (false) {}
|
if (false) {}
|
||||||
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
|
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
|
||||||
@ -1494,11 +1494,11 @@ void NO_INLINE Aggregator::mergeWithoutKeyDataImpl(
|
|||||||
{
|
{
|
||||||
AggregatedDataVariantsPtr & res = non_empty_data[0];
|
AggregatedDataVariantsPtr & res = non_empty_data[0];
|
||||||
|
|
||||||
/// We connect all aggregation results to the first.
|
/// We merge all aggregation results to the first.
|
||||||
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
|
for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num)
|
||||||
{
|
{
|
||||||
AggregatedDataWithoutKey & res_data = res->without_key;
|
AggregatedDataWithoutKey & res_data = res->without_key;
|
||||||
AggregatedDataWithoutKey & current_data = non_empty_data[i]->without_key;
|
AggregatedDataWithoutKey & current_data = non_empty_data[result_num]->without_key;
|
||||||
|
|
||||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||||
aggregate_functions[i]->merge(res_data + offsets_of_aggregate_states[i], current_data + offsets_of_aggregate_states[i], res->aggregates_pool);
|
aggregate_functions[i]->merge(res_data + offsets_of_aggregate_states[i], current_data + offsets_of_aggregate_states[i], res->aggregates_pool);
|
||||||
@ -1518,13 +1518,13 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl(
|
|||||||
AggregatedDataVariantsPtr & res = non_empty_data[0];
|
AggregatedDataVariantsPtr & res = non_empty_data[0];
|
||||||
bool no_more_keys = false;
|
bool no_more_keys = false;
|
||||||
|
|
||||||
/// We connect all aggregation results to the first.
|
/// We merge all aggregation results to the first.
|
||||||
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
|
for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num)
|
||||||
{
|
{
|
||||||
if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys))
|
if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys))
|
||||||
break;
|
break;
|
||||||
|
|
||||||
AggregatedDataVariants & current = *non_empty_data[i];
|
AggregatedDataVariants & current = *non_empty_data[result_num];
|
||||||
|
|
||||||
if (!no_more_keys)
|
if (!no_more_keys)
|
||||||
mergeDataImpl<Method>(
|
mergeDataImpl<Method>(
|
||||||
@ -1553,11 +1553,11 @@ template <typename Method>
|
|||||||
void NO_INLINE Aggregator::mergeBucketImpl(
|
void NO_INLINE Aggregator::mergeBucketImpl(
|
||||||
ManyAggregatedDataVariants & data, Int32 bucket, Arena * arena) const
|
ManyAggregatedDataVariants & data, Int32 bucket, Arena * arena) const
|
||||||
{
|
{
|
||||||
/// We connect all aggregation results to the first.
|
/// We merge all aggregation results to the first.
|
||||||
AggregatedDataVariantsPtr & res = data[0];
|
AggregatedDataVariantsPtr & res = data[0];
|
||||||
for (size_t i = 1, size = data.size(); i < size; ++i)
|
for (size_t result_num = 1, size = data.size(); result_num < size; ++result_num)
|
||||||
{
|
{
|
||||||
AggregatedDataVariants & current = *data[i];
|
AggregatedDataVariants & current = *data[result_num];
|
||||||
|
|
||||||
mergeDataImpl<Method>(
|
mergeDataImpl<Method>(
|
||||||
getDataVariant<Method>(*res).data.impls[bucket],
|
getDataVariant<Method>(*res).data.impls[bucket],
|
||||||
|
@ -231,11 +231,11 @@ ProcessListEntry::~ProcessListEntry()
|
|||||||
auto range = user_process_list.queries.equal_range(query_id);
|
auto range = user_process_list.queries.equal_range(query_id);
|
||||||
if (range.first != range.second)
|
if (range.first != range.second)
|
||||||
{
|
{
|
||||||
for (auto it = range.first; it != range.second; ++it)
|
for (auto jt = range.first; jt != range.second; ++jt)
|
||||||
{
|
{
|
||||||
if (it->second == process_list_element_ptr)
|
if (jt->second == process_list_element_ptr)
|
||||||
{
|
{
|
||||||
user_process_list.queries.erase(it);
|
user_process_list.queries.erase(jt);
|
||||||
found = true;
|
found = true;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -269,7 +269,6 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
res.finish_callback = [elem, &context, log_queries] (IBlockInputStream * stream_in, IBlockOutputStream * stream_out) mutable
|
res.finish_callback = [elem, &context, log_queries] (IBlockInputStream * stream_in, IBlockOutputStream * stream_out) mutable
|
||||||
{
|
{
|
||||||
QueryStatus * process_list_elem = context.getProcessListElement();
|
QueryStatus * process_list_elem = context.getProcessListElement();
|
||||||
const Settings & settings = context.getSettingsRef();
|
|
||||||
|
|
||||||
if (!process_list_elem)
|
if (!process_list_elem)
|
||||||
return;
|
return;
|
||||||
@ -277,7 +276,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
/// Update performance counters before logging to query_log
|
/// Update performance counters before logging to query_log
|
||||||
CurrentThread::finalizePerformanceCounters();
|
CurrentThread::finalizePerformanceCounters();
|
||||||
|
|
||||||
QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events);
|
QueryStatusInfo info = process_list_elem->getInfo(true, context.getSettingsRef().log_profile_events);
|
||||||
|
|
||||||
double elapsed_seconds = info.elapsed_seconds;
|
double elapsed_seconds = info.elapsed_seconds;
|
||||||
|
|
||||||
@ -345,14 +344,14 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
elem.exception = getCurrentExceptionMessage(false);
|
elem.exception = getCurrentExceptionMessage(false);
|
||||||
|
|
||||||
QueryStatus * process_list_elem = context.getProcessListElement();
|
QueryStatus * process_list_elem = context.getProcessListElement();
|
||||||
const Settings & settings = context.getSettingsRef();
|
const Settings & current_settings = context.getSettingsRef();
|
||||||
|
|
||||||
/// Update performance counters before logging to query_log
|
/// Update performance counters before logging to query_log
|
||||||
CurrentThread::finalizePerformanceCounters();
|
CurrentThread::finalizePerformanceCounters();
|
||||||
|
|
||||||
if (process_list_elem)
|
if (process_list_elem)
|
||||||
{
|
{
|
||||||
QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events, false);
|
QueryStatusInfo info = process_list_elem->getInfo(true, current_settings.log_profile_events, false);
|
||||||
|
|
||||||
elem.query_duration_ms = info.elapsed_seconds * 1000;
|
elem.query_duration_ms = info.elapsed_seconds * 1000;
|
||||||
|
|
||||||
@ -365,7 +364,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
elem.profile_counters = std::move(info.profile_counters);
|
elem.profile_counters = std::move(info.profile_counters);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (settings.calculate_text_stack_trace)
|
if (current_settings.calculate_text_stack_trace)
|
||||||
setExceptionStackTrace(elem);
|
setExceptionStackTrace(elem);
|
||||||
logException(context, elem);
|
logException(context, elem);
|
||||||
|
|
||||||
|
@ -136,7 +136,7 @@ void sortBlock(Block & block, const SortDescription & description, size_t limit)
|
|||||||
bool need_collation = false;
|
bool need_collation = false;
|
||||||
ColumnsWithSortDescriptions columns_with_sort_desc = getColumnsWithSortDescription(block, description);
|
ColumnsWithSortDescriptions columns_with_sort_desc = getColumnsWithSortDescription(block, description);
|
||||||
|
|
||||||
for (size_t i = 0, size = description.size(); i < size; ++i)
|
for (size_t i = 0, num_sort_columns = description.size(); i < num_sort_columns; ++i)
|
||||||
{
|
{
|
||||||
if (needCollation(columns_with_sort_desc[i].first, description[i]))
|
if (needCollation(columns_with_sort_desc[i].first, description[i]))
|
||||||
{
|
{
|
||||||
|
Loading…
Reference in New Issue
Block a user