Fix #ci_set_analyzer

This commit is contained in:
Dmitry Novik 2024-03-19 19:44:15 +01:00
parent 302179c81d
commit 221b74c77f
5 changed files with 22 additions and 0 deletions

View File

@ -99,6 +99,11 @@ public:
return settings_changes;
}
void clearSettingsChanges()
{
settings_changes.clear();
}
/// Returns true if query node is subquery, false otherwise
bool isSubquery() const
{

View File

@ -127,6 +127,7 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster,
///
/// Here we don't try to analyze setting again. In case if query_info->additional_filter_ast is not empty, some filter was applied.
/// It's just easier to add this filter for a source table.
LOG_DEBUG(&Poco::Logger::get("updateSettings"), "{}", additional_filter_ast != nullptr);
if (additional_filter_ast)
{
Tuple tuple;

View File

@ -1,5 +1,6 @@
#include <Planner/PlannerJoinTree.h>
#include "Common/logger_useful.h"
#include <Common/scope_guard_safe.h>
#include <Columns/ColumnAggregateFunction.h>
@ -63,6 +64,7 @@
#include <Planner/Utils.h>
#include <Planner/CollectSets.h>
#include <Planner/CollectTableExpressionData.h>
#include <Poco/Logger.h>
namespace DB
{
@ -523,6 +525,7 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage,
return {};
auto const & storage_id = storage->getStorageID();
LOG_DEBUG(&Poco::Logger::get("Planner"), "{}", storage_id.getFullNameNotQuoted());
ASTPtr additional_filter_ast;
for (const auto & additional_filter : additional_filters)
@ -543,6 +546,8 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage,
}
}
LOG_DEBUG(&Poco::Logger::get("Planner"), "{}", additional_filter_ast != nullptr);
if (!additional_filter_ast)
return {};
@ -847,6 +852,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
const auto & table_expression_alias = table_expression->getOriginalAlias();
auto additional_filters_info
= buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context);
if (additional_filters_info.actions)
LOG_DEBUG(&Poco::Logger::get("Planner"), "{}", additional_filters_info.actions->dumpDAG());
add_filter(additional_filters_info, "additional filter");
from_stage = storage->getQueryProcessingStage(

View File

@ -836,6 +836,8 @@ void StorageDistributed::read(
{
Block header;
LOG_DEBUG(&Poco::Logger::get("read"), "{}", query_info.additional_filter_ast != nullptr);
SelectQueryInfo modified_query_info = query_info;
if (local_context->getSettingsRef().allow_experimental_analyzer)

View File

@ -20,6 +20,7 @@
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include "Analyzer/QueryNode.h"
namespace DB
{
@ -383,6 +384,12 @@ QueryTreeNodePtr buildQueryTreeForShard(const PlannerContextPtr & planner_contex
createUniqueTableAliases(query_tree_to_modify, nullptr, planner_context->getQueryContext());
// Get rid of the settings clause so we don't send them to remote. Thus newly non-important
// settings won't break any remote parser. It's also more reasonable since the query settings
// are written into the query context and will be sent by the query pipeline.
if (auto * query_node = query_tree_to_modify->as<QueryNode>())
query_node->clearSettingsChanges();
return query_tree_to_modify;
}