mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #63429 from Algunenano/micro_analyzer
Micro-optimizations for the new analyzer
This commit is contained in:
commit
ad5403a034
@ -89,7 +89,7 @@ PenaltyBreakFirstLessLess: 120
|
||||
PenaltyBreakString: 1000
|
||||
PenaltyExcessCharacter: 1000000
|
||||
PenaltyReturnTypeOnItsOwnLine: 60
|
||||
RemoveBracesLLVM: true
|
||||
RemoveBracesLLVM: false
|
||||
SpaceAfterCStyleCast: false
|
||||
SpaceBeforeAssignmentOperators: true
|
||||
SpaceBeforeParens: ControlStatements
|
||||
|
@ -138,6 +138,8 @@ Checks: [
|
||||
|
||||
# This is a good check, but clang-tidy crashes, see https://github.com/llvm/llvm-project/issues/91872
|
||||
'-modernize-use-constraints',
|
||||
# https://github.com/abseil/abseil-cpp/issues/1667
|
||||
'-clang-analyzer-optin.core.EnumCastOutOfRange'
|
||||
]
|
||||
|
||||
WarningsAsErrors: '*'
|
||||
|
@ -21,6 +21,9 @@
|
||||
#include <base/sort.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
|
||||
#include <absl/container/flat_hash_map.h>
|
||||
#include <absl/container/inlined_vector.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -708,16 +711,18 @@ static ColumnWithTypeAndName executeActionForPartialResult(const ActionsDAG::Nod
|
||||
return res_column;
|
||||
}
|
||||
|
||||
Block ActionsDAG::updateHeader(Block header) const
|
||||
Block ActionsDAG::updateHeader(const Block & header) const
|
||||
{
|
||||
IntermediateExecutionResult node_to_column;
|
||||
std::set<size_t> pos_to_remove;
|
||||
|
||||
{
|
||||
std::unordered_map<std::string_view, std::list<size_t>> input_positions;
|
||||
using inline_vector = absl::InlinedVector<size_t, 7>; // 64B, holding max 7 size_t elements inlined
|
||||
absl::flat_hash_map<std::string_view, inline_vector> input_positions;
|
||||
|
||||
for (size_t pos = 0; pos < inputs.size(); ++pos)
|
||||
input_positions[inputs[pos]->result_name].emplace_back(pos);
|
||||
/// We insert from last to first in the inlinedVector so it's easier to pop_back matches later
|
||||
for (size_t pos = inputs.size(); pos != 0; pos--)
|
||||
input_positions[inputs[pos - 1]->result_name].emplace_back(pos - 1);
|
||||
|
||||
for (size_t pos = 0; pos < header.columns(); ++pos)
|
||||
{
|
||||
@ -725,10 +730,11 @@ Block ActionsDAG::updateHeader(Block header) const
|
||||
auto it = input_positions.find(col.name);
|
||||
if (it != input_positions.end() && !it->second.empty())
|
||||
{
|
||||
auto & list = it->second;
|
||||
pos_to_remove.insert(pos);
|
||||
node_to_column[inputs[list.front()]] = col;
|
||||
list.pop_front();
|
||||
|
||||
auto & v = it->second;
|
||||
node_to_column[inputs[v.back()]] = col;
|
||||
v.pop_back();
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -746,18 +752,21 @@ Block ActionsDAG::updateHeader(Block header) const
|
||||
throw;
|
||||
}
|
||||
|
||||
if (isInputProjected())
|
||||
header.clear();
|
||||
else
|
||||
header.erase(pos_to_remove);
|
||||
|
||||
Block res;
|
||||
|
||||
res.reserve(result_columns.size());
|
||||
for (auto & col : result_columns)
|
||||
res.insert(std::move(col));
|
||||
|
||||
for (auto && item : header)
|
||||
res.insert(std::move(item));
|
||||
if (isInputProjected())
|
||||
return res;
|
||||
|
||||
res.reserve(header.columns() - pos_to_remove.size());
|
||||
for (size_t i = 0; i < header.columns(); i++)
|
||||
{
|
||||
if (!pos_to_remove.contains(i))
|
||||
res.insert(header.data[i]);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
@ -272,7 +272,7 @@ public:
|
||||
///
|
||||
/// In addition, check that result constants are constants according to DAG.
|
||||
/// In case if function return constant, but arguments are not constant, materialize it.
|
||||
Block updateHeader(Block header) const;
|
||||
Block updateHeader(const Block & header) const;
|
||||
|
||||
using IntermediateExecutionResult = std::unordered_map<const Node *, ColumnWithTypeAndName>;
|
||||
static ColumnsWithTypeAndName evaluatePartialResult(
|
||||
|
@ -2487,10 +2487,15 @@ HashJoin::~HashJoin()
|
||||
{
|
||||
if (!data)
|
||||
{
|
||||
LOG_TRACE(log, "{}Join data has been already released", instance_log_id);
|
||||
LOG_TEST(log, "{}Join data has been already released", instance_log_id);
|
||||
return;
|
||||
}
|
||||
LOG_TRACE(log, "{}Join data is being destroyed, {} bytes and {} rows in hash table", instance_log_id, getTotalByteCount(), getTotalRowCount());
|
||||
LOG_TEST(
|
||||
log,
|
||||
"{}Join data is being destroyed, {} bytes and {} rows in hash table",
|
||||
instance_log_id,
|
||||
getTotalByteCount(),
|
||||
getTotalRowCount());
|
||||
}
|
||||
|
||||
template <typename Mapped>
|
||||
|
@ -1229,8 +1229,9 @@ void Planner::buildQueryPlanIfNeeded()
|
||||
if (query_plan.isInitialized())
|
||||
return;
|
||||
|
||||
LOG_TRACE(getLogger("Planner"), "Query {} to stage {}{}",
|
||||
query_tree->formatConvertedASTForErrorMessage(),
|
||||
LOG_TRACE(
|
||||
getLogger("Planner"),
|
||||
"Query to stage {}{}",
|
||||
QueryProcessingStage::toString(select_query_options.to_stage),
|
||||
select_query_options.only_analyze ? " only analyze" : "");
|
||||
|
||||
@ -1506,8 +1507,9 @@ void Planner::buildPlanForQueryNode()
|
||||
auto & mapping = join_tree_query_plan.query_node_to_plan_step_mapping;
|
||||
query_node_to_plan_step_mapping.insert(mapping.begin(), mapping.end());
|
||||
|
||||
LOG_TRACE(getLogger("Planner"), "Query {} from stage {} to stage {}{}",
|
||||
query_tree->formatConvertedASTForErrorMessage(),
|
||||
LOG_TRACE(
|
||||
getLogger("Planner"),
|
||||
"Query from stage {} to stage {}{}",
|
||||
QueryProcessingStage::toString(from_stage),
|
||||
QueryProcessingStage::toString(select_query_options.to_stage),
|
||||
select_query_options.only_analyze ? " only analyze" : "");
|
||||
|
@ -21,7 +21,7 @@ Block SourceStepWithFilter::applyPrewhereActions(Block block, const PrewhereInfo
|
||||
{
|
||||
if (prewhere_info->row_level_filter)
|
||||
{
|
||||
block = prewhere_info->row_level_filter->updateHeader(std::move(block));
|
||||
block = prewhere_info->row_level_filter->updateHeader(block);
|
||||
auto & row_level_column = block.getByName(prewhere_info->row_level_column_name);
|
||||
if (!row_level_column.type->canBeUsedInBooleanContext())
|
||||
{
|
||||
@ -36,7 +36,7 @@ Block SourceStepWithFilter::applyPrewhereActions(Block block, const PrewhereInfo
|
||||
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
block = prewhere_info->prewhere_actions->updateHeader(std::move(block));
|
||||
block = prewhere_info->prewhere_actions->updateHeader(block);
|
||||
|
||||
auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name);
|
||||
if (!prewhere_column.type->canBeUsedInBooleanContext())
|
||||
|
@ -3,9 +3,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
Block ExpressionTransform::transformHeader(Block header, const ActionsDAG & expression)
|
||||
Block ExpressionTransform::transformHeader(const Block & header, const ActionsDAG & expression)
|
||||
{
|
||||
return expression.updateHeader(std::move(header));
|
||||
return expression.updateHeader(header);
|
||||
}
|
||||
|
||||
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
|
||||
String getName() const override { return "ExpressionTransform"; }
|
||||
|
||||
static Block transformHeader(Block header, const ActionsDAG & expression);
|
||||
static Block transformHeader(const Block & header, const ActionsDAG & expression);
|
||||
|
||||
protected:
|
||||
void transform(Chunk & chunk) override;
|
||||
|
@ -174,26 +174,22 @@ static std::unique_ptr<IFilterDescription> combineFilterAndIndices(
|
||||
}
|
||||
|
||||
Block FilterTransform::transformHeader(
|
||||
Block header,
|
||||
const ActionsDAG * expression,
|
||||
const String & filter_column_name,
|
||||
bool remove_filter_column)
|
||||
const Block & header, const ActionsDAG * expression, const String & filter_column_name, bool remove_filter_column)
|
||||
{
|
||||
if (expression)
|
||||
header = expression->updateHeader(std::move(header));
|
||||
Block result = expression ? expression->updateHeader(header) : header;
|
||||
|
||||
auto filter_type = header.getByName(filter_column_name).type;
|
||||
auto filter_type = result.getByName(filter_column_name).type;
|
||||
if (!filter_type->onlyNull() && !isUInt8(removeNullable(removeLowCardinality(filter_type))))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER,
|
||||
"Illegal type {} of column {} for filter. Must be UInt8 or Nullable(UInt8).",
|
||||
filter_type->getName(), filter_column_name);
|
||||
|
||||
if (remove_filter_column)
|
||||
header.erase(filter_column_name);
|
||||
result.erase(filter_column_name);
|
||||
else
|
||||
replaceFilterToConstant(header, filter_column_name);
|
||||
replaceFilterToConstant(result, filter_column_name);
|
||||
|
||||
return header;
|
||||
return result;
|
||||
}
|
||||
|
||||
FilterTransform::FilterTransform(
|
||||
|
@ -22,11 +22,8 @@ public:
|
||||
const Block & header_, ExpressionActionsPtr expression_, String filter_column_name_,
|
||||
bool remove_filter_column_, bool on_totals_ = false, std::shared_ptr<std::atomic<size_t>> rows_filtered_ = nullptr);
|
||||
|
||||
static Block transformHeader(
|
||||
Block header,
|
||||
const ActionsDAG * expression,
|
||||
const String & filter_column_name,
|
||||
bool remove_filter_column);
|
||||
static Block
|
||||
transformHeader(const Block & header, const ActionsDAG * expression, const String & filter_column_name, bool remove_filter_column);
|
||||
|
||||
String getName() const override { return "FilterTransform"; }
|
||||
|
||||
|
@ -14,12 +14,12 @@ namespace ErrorCodes
|
||||
|
||||
Block JoiningTransform::transformHeader(Block header, const JoinPtr & join)
|
||||
{
|
||||
LOG_DEBUG(getLogger("JoiningTransform"), "Before join block: '{}'", header.dumpStructure());
|
||||
LOG_TEST(getLogger("JoiningTransform"), "Before join block: '{}'", header.dumpStructure());
|
||||
join->checkTypesOfKeys(header);
|
||||
join->initialize(header);
|
||||
ExtraBlockPtr tmp;
|
||||
join->joinBlock(header, tmp);
|
||||
LOG_DEBUG(getLogger("JoiningTransform"), "After join block: '{}'", header.dumpStructure());
|
||||
LOG_TEST(getLogger("JoiningTransform"), "After join block: '{}'", header.dumpStructure());
|
||||
return header;
|
||||
}
|
||||
|
||||
|
@ -49,7 +49,7 @@ Block TotalsHavingTransform::transformHeader(
|
||||
|
||||
if (expression)
|
||||
{
|
||||
block = expression->updateHeader(std::move(block));
|
||||
block = expression->updateHeader(block);
|
||||
if (remove_filter)
|
||||
block.erase(filter_column_name);
|
||||
}
|
||||
|
@ -207,7 +207,7 @@ select
|
||||
with 0.16 as threshold
|
||||
select
|
||||
'noisy Trace messages',
|
||||
greatest(coalesce(((select message_format_string, count() from logs where level = 'Trace' and message_format_string not in ('Access granted: {}{}', '{} -> {}', 'Query {} to stage {}{}', 'Query {} from stage {} to stage {}{}')
|
||||
greatest(coalesce(((select message_format_string, count() from logs where level = 'Trace' and message_format_string not in ('Access granted: {}{}', '{} -> {}', 'Query to stage {}{}', 'Query from stage {} to stage {}{}')
|
||||
group by message_format_string order by count() desc limit 1) as top_message).2, 0) / (select count() from logs), threshold) as r,
|
||||
r <= threshold ? '' : top_message.1;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user