Fixed build and made setting an alias

This commit is contained in:
Nikita Mikhaylov 2024-07-29 17:02:11 +00:00
parent 41bfdbe64e
commit a8ca5ad50b
34 changed files with 82 additions and 81 deletions

View File

@ -196,11 +196,11 @@ void HedgedConnections::sendQuery(
modified_settings.parallel_replica_offset = fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset;
}
/// FIXME: Remove once we will make `enable_analyzer` obsolete setting.
/// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting.
/// Make the analyzer being set, so it will be effectively applied on the remote server.
/// In other words, the initiator always controls whether the analyzer enabled or not for
/// all servers involved in the distributed query processing.
modified_settings.set("enable_analyzer", static_cast<bool>(modified_settings.enable_analyzer));
modified_settings.set("allow_experimental_analyzer", static_cast<bool>(modified_settings.allow_experimental_analyzer));
replica.connection->sendQuery(
timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {});

View File

@ -150,11 +150,11 @@ void MultiplexedConnections::sendQuery(
client_info.number_of_current_replica = replica_info->number_of_current_replica;
}
/// FIXME: Remove once we will make `enable_analyzer` obsolete setting.
/// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting.
/// Make the analyzer being set, so it will be effectively applied on the remote server.
/// In other words, the initiator always controls whether the analyzer enabled or not for
/// all servers involved in the distributed query processing.
modified_settings.set("enable_analyzer", static_cast<bool>(modified_settings.enable_analyzer));
modified_settings.set("allow_experimental_analyzer", static_cast<bool>(modified_settings.allow_experimental_analyzer));
const bool enable_offset_parallel_processing = context->canUseOffsetParallelReplicas();

View File

@ -638,7 +638,7 @@ class IColumn;
M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \
M(Bool, enable_global_with_statement, true, "Propagate WITH statements to UNION queries and all subqueries", 0) \
M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \
M(Bool, optimize_syntax_fuse_functions, false, "Allow apply fuse aggregating function. Available only with `enable_analyzer`", 0) \
M(Bool, optimize_syntax_fuse_functions, false, "Allow apply fuse aggregating function. Available only with `allow_experimental_analyzer`", 0) \
M(Bool, flatten_nested, true, "If true, columns of type Nested will be flatten to separate array columns instead of one array of tuples", 0) \
M(Bool, asterisk_include_materialized_columns, false, "Include MATERIALIZED columns for wildcard query", 0) \
M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \
@ -943,7 +943,7 @@ class IColumn;
\
M(Bool, allow_experimental_join_condition, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y.", 0) \
\
M(Bool, enable_analyzer, true, "Allow new query analyzer.", IMPORTANT) ALIAS(allow_experimental_analyzer) \
M(Bool, allow_experimental_analyzer, true, "Allow new query analyzer.", IMPORTANT) ALIAS(enable_analyzer) \
M(Bool, analyzer_compatibility_join_using_top_level_identifier, false, "Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`).", 0) \
\
M(Bool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \

View File

@ -528,6 +528,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"restore_replace_external_table_functions_to_null", false, false, "New setting."},
{"restore_replace_external_engines_to_null", false, false, "New setting."}
{"enable_analyzer", false, true, "Move analyzer to the Beta stage."},
{"allow_experimental_analyzer", false, true, "Move analyzer to the Beta stage."},
}},
{"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"},
{"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"},

View File

@ -1411,7 +1411,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool
return {};
PreparedSets::Hash set_key;
if (data.getContext()->getSettingsRef().enable_analyzer && !identifier)
if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier)
{
/// Here we can be only from mutation interpreter. Normal selects with analyzed use other interpreter.
/// This is a hacky way to allow reusing cache for prepared sets.

View File

@ -68,7 +68,7 @@ ASTPtr rewriteSelectQuery(
// are written into the query context and will be sent by the query pipeline.
select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, {});
if (!context->getSettingsRef().enable_analyzer)
if (!context->getSettingsRef().allow_experimental_analyzer)
{
if (table_function_ptr)
select_query.addTableFunction(table_function_ptr);
@ -165,7 +165,7 @@ void SelectStreamFactory::createForShardImpl(
auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0)
{
Block shard_header;
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
shard_header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree, context, SelectQueryOptions(processed_stage).analyze());
else
shard_header = header;

View File

@ -300,7 +300,7 @@ void executeQuery(
const size_t shards = cluster->getShardCount();
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
{
for (size_t i = 0, s = cluster->getShardsInfo().size(); i < s; ++i)
{
@ -581,7 +581,7 @@ void executeQueryWithParallelReplicasCustomKey(
/// Return directly (with correct header) if no shard to query.
if (query_info.getCluster()->getShardsInfo().empty())
{
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
return;
Pipe pipe(std::make_shared<NullSource>(header));

View File

@ -834,7 +834,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
Block as_select_sample;
if (getContext()->getSettingsRef().enable_analyzer)
if (getContext()->getSettingsRef().allow_experimental_analyzer)
{
as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext());
}
@ -1327,7 +1327,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
{
Block input_block;
if (getContext()->getSettingsRef().enable_analyzer)
if (getContext()->getSettingsRef().allow_experimental_analyzer)
{
input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext());
}

View File

@ -129,7 +129,7 @@ void InterpreterDescribeQuery::fillColumnsFromSubquery(const ASTTableExpression
auto select_query = table_expression.subquery->children.at(0);
auto current_context = getContext();
if (settings.enable_analyzer)
if (settings.allow_experimental_analyzer)
{
SelectQueryOptions select_query_options;
sample_block = InterpreterSelectQueryAnalyzer(select_query, current_context, select_query_options).getSampleBlock();

View File

@ -394,9 +394,9 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
}
case ASTExplainQuery::QueryTree:
{
if (!getContext()->getSettingsRef().enable_analyzer)
if (!getContext()->getSettingsRef().allow_experimental_analyzer)
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"EXPLAIN QUERY TREE is only supported with a new analyzer. Set enable_analyzer = 1.");
"EXPLAIN QUERY TREE is only supported with a new analyzer. Set allow_experimental_analyzer = 1.");
if (ast.getExplainedQuery()->as<ASTSelectWithUnionQuery>() == nullptr)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN QUERY TREE query");
@ -453,7 +453,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
ContextPtr context;
if (getContext()->getSettingsRef().enable_analyzer)
if (getContext()->getSettingsRef().allow_experimental_analyzer)
{
InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), options);
context = interpreter.getContext();
@ -499,7 +499,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
QueryPlan plan;
ContextPtr context;
if (getContext()->getSettingsRef().enable_analyzer)
if (getContext()->getSettingsRef().allow_experimental_analyzer)
{
InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), options);
context = interpreter.getContext();
@ -558,7 +558,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
QueryPlan plan;
ContextPtr context = getContext();
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
{
InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions());
context = interpreter.getContext();

View File

@ -118,7 +118,7 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte
if (query->as<ASTSelectQuery>())
{
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
interpreter_name = "InterpreterSelectQueryAnalyzer";
/// This is internal part of ASTSelectWithUnionQuery.
/// Even if there is SELECT without union, it is represented by ASTSelectWithUnionQuery with single ASTSelectQuery as a child.
@ -129,7 +129,7 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte
{
ProfileEvents::increment(ProfileEvents::SelectQuery);
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
interpreter_name = "InterpreterSelectQueryAnalyzer";
else
interpreter_name = "InterpreterSelectWithUnionQuery";
@ -222,7 +222,7 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte
{
const auto kind = query->as<ASTExplainQuery>()->getKind();
if (kind == ASTExplainQuery::ParsedAST || kind == ASTExplainQuery::AnalyzedSyntax)
context->setSetting("enable_analyzer", false);
context->setSetting("allow_experimental_analyzer", false);
interpreter_name = "InterpreterExplainQuery";
}

View File

@ -95,7 +95,7 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query)
Block header_block;
auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1);
if (current_context->getSettingsRef().enable_analyzer)
if (current_context->getSettingsRef().allow_experimental_analyzer)
{
InterpreterSelectQueryAnalyzer interpreter_select(query.select, current_context, select_query_options);
header_block = interpreter_select.getSampleBlock();

View File

@ -189,7 +189,7 @@ bool isStorageTouchedByMutations(
std::optional<InterpreterSelectQuery> interpreter_select_query;
BlockIO io;
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
{
auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage.shared_from_this(), context);
InterpreterSelectQueryAnalyzer interpreter(select_query_tree, context, SelectQueryOptions().ignoreLimits());
@ -415,9 +415,9 @@ MutationsInterpreter::MutationsInterpreter(
, logger(getLogger("MutationsInterpreter(" + source.getStorage()->getStorageID().getFullTableName() + ")"))
{
auto new_context = Context::createCopy(context_);
if (new_context->getSettingsRef().enable_analyzer)
if (new_context->getSettingsRef().allow_experimental_analyzer)
{
new_context->setSetting("enable_analyzer", false);
new_context->setSetting("allow_experimental_analyzer", false);
LOG_DEBUG(logger, "Will use old analyzer to prepare mutation");
}
context = std::move(new_context);

View File

@ -685,10 +685,10 @@ void validateAnalyzerSettings(ASTPtr ast, bool context_value)
if (auto * set_query = node->as<ASTSetQuery>())
{
if (auto * value = set_query->changes.tryGet("enable_analyzer"))
if (auto * value = set_query->changes.tryGet("allow_experimental_analyzer"))
{
if (top_level != value->safeGet<bool>())
throw Exception(ErrorCodes::INCORRECT_QUERY, "Setting 'enable_analyzer' is changed in the subquery. Top level value: {}", top_level);
throw Exception(ErrorCodes::INCORRECT_QUERY, "Setting 'allow_experimental_analyzer' is changed in the subquery. Top level value: {}", top_level);
}
}
@ -912,7 +912,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Interpret SETTINGS clauses as early as possible (before invoking the corresponding interpreter),
/// to allow settings to take effect.
InterpreterSetQuery::applySettingsFromQuery(ast, context);
validateAnalyzerSettings(ast, context->getSettingsRef().enable_analyzer);
validateAnalyzerSettings(ast, context->getSettingsRef().allow_experimental_analyzer);
if (auto * insert_query = ast->as<ASTInsertQuery>())
insert_query->tail = istr;

View File

@ -141,7 +141,7 @@ Block getHeaderForProcessingStage(
Block result;
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
{
auto storage = std::make_shared<StorageDummy>(storage_snapshot->storage.getStorageID(),
storage_snapshot->getAllColumnsDescription(),

View File

@ -65,7 +65,7 @@ std::unique_ptr<QueryPlan> createLocalPlan(
.setShardInfo(static_cast<UInt32>(shard_num), static_cast<UInt32>(shard_count))
.ignoreASTOptimizations();
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
{
/// For Analyzer, identifier in GROUP BY/ORDER BY/LIMIT BY lists has been resolved to
/// ConstantNode in QueryTree if it is an alias of a constant, so we should not replace

View File

@ -319,7 +319,7 @@ std::optional<Chain> generateViewChain(
Block header;
/// Get list of columns we get from select query.
if (select_context->getSettingsRef().enable_analyzer)
if (select_context->getSettingsRef().allow_experimental_analyzer)
header = InterpreterSelectQueryAnalyzer::getSampleBlock(query, select_context);
else
header = InterpreterSelectQuery(query, select_context, SelectQueryOptions()).getSampleBlock();
@ -613,7 +613,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat
QueryPipelineBuilder pipeline;
if (local_context->getSettingsRef().enable_analyzer)
if (local_context->getSettingsRef().allow_experimental_analyzer)
{
InterpreterSelectQueryAnalyzer interpreter(view.query, local_context, local_context->getViewSource(), SelectQueryOptions().ignoreAccessCheck());
pipeline = interpreter.buildQueryPipeline();

View File

@ -1904,14 +1904,14 @@ void TCPHandler::receiveQuery()
/// Settings
///
/// FIXME: Remove when enable_analyzer will become obsolete.
/// FIXME: Remove when allow_experimental_analyzer will become obsolete.
/// Analyzer became Beta in 24.3 and started to be enabled by default.
/// We have to disable it for ourselves to make sure we don't have different settings on
/// different servers.
if (query_kind == ClientInfo::QueryKind::SECONDARY_QUERY
&& client_info.getVersionNumber() < VersionNumber(23, 3, 0)
&& !passed_settings.enable_analyzer.changed)
passed_settings.set("enable_analyzer", false);
&& !passed_settings.allow_experimental_analyzer.changed)
passed_settings.set("allow_experimental_analyzer", false);
auto settings_changes = passed_settings.changes();
query_kind = query_context->getClientInfo().query_kind;

View File

@ -806,7 +806,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
metadata.select = SelectQueryDescription::getSelectQueryFromASTForMatView(select, metadata.refresh != nullptr, context);
Block as_select_sample;
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
{
as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(select->clone(), context);
}

View File

@ -125,7 +125,7 @@ void IStorageCluster::read(
Block sample_block;
ASTPtr query_to_send = query_info.query;
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
{
sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(query_info.query, context, SelectQueryOptions(processed_stage));
}

View File

@ -378,7 +378,7 @@ void StorageLiveView::writeBlock(StorageLiveView & live_view, Block && block, Ch
QueryPipelineBuilder builder;
if (local_context->getSettingsRef().enable_analyzer)
if (local_context->getSettingsRef().allow_experimental_analyzer)
{
auto select_description = buildSelectQueryTreeDescription(select_query_description.inner_query, local_context);
if (select_description.dependent_table_node)
@ -475,7 +475,7 @@ Block StorageLiveView::getHeader() const
if (!sample_block)
{
if (live_view_context->getSettingsRef().enable_analyzer)
if (live_view_context->getSettingsRef().allow_experimental_analyzer)
{
sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(select_query_description.select_query,
live_view_context,
@ -519,7 +519,7 @@ ASTPtr StorageLiveView::getInnerBlocksQuery()
auto & select_with_union_query = select_query_description.select_query->as<ASTSelectWithUnionQuery &>();
auto blocks_query = select_with_union_query.list_of_selects->children.at(0)->clone();
if (!live_view_context->getSettingsRef().enable_analyzer)
if (!live_view_context->getSettingsRef().allow_experimental_analyzer)
{
/// Rewrite inner query with right aliases for JOIN.
/// It cannot be done in constructor or startup() because InterpreterSelectQuery may access table,
@ -543,7 +543,7 @@ MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(ContextPtr local_cont
QueryPipelineBuilder builder;
if (local_context->getSettingsRef().enable_analyzer)
if (local_context->getSettingsRef().allow_experimental_analyzer)
{
InterpreterSelectQueryAnalyzer interpreter(select_query_description.inner_query,
local_context,
@ -599,7 +599,7 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes)
QueryPipelineBuilder builder;
if (block_context->getSettingsRef().enable_analyzer)
if (block_context->getSettingsRef().allow_experimental_analyzer)
{
auto select_description = buildSelectQueryTreeDescription(select_query_description.select_query, block_context);

View File

@ -7097,7 +7097,7 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage(
SelectQueryInfo &) const
{
/// with new analyzer, Planner make decision regarding parallel replicas usage, and so about processing stage on reading
if (!query_context->getSettingsRef().enable_analyzer)
if (!query_context->getSettingsRef().allow_experimental_analyzer)
{
const auto & settings = query_context->getSettingsRef();
if (query_context->canUseParallelReplicasCustomKey())

View File

@ -33,7 +33,7 @@ namespace ErrorCodes
namespace
{
void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool enable_analyzer, bool legacy = false)
void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool allow_experimental_analyzer, bool legacy = false)
{
switch (node.type)
{
@ -45,18 +45,18 @@ void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & o
/// If it was created from ASTLiteral, then result_name can be an alias.
/// We need to convert value back to string here.
const auto * column_const = typeid_cast<const ColumnConst *>(node.column.get());
if (column_const && !enable_analyzer)
if (column_const && !allow_experimental_analyzer)
writeString(applyVisitor(FieldVisitorToString(), column_const->getField()), out);
else
writeString(node.result_name, out);
break;
}
case ActionsDAG::ActionType::ALIAS:
appendColumnNameWithoutAlias(*node.children.front(), out, enable_analyzer, legacy);
appendColumnNameWithoutAlias(*node.children.front(), out, allow_experimental_analyzer, legacy);
break;
case ActionsDAG::ActionType::ARRAY_JOIN:
writeCString("arrayJoin(", out);
appendColumnNameWithoutAlias(*node.children.front(), out, enable_analyzer, legacy);
appendColumnNameWithoutAlias(*node.children.front(), out, allow_experimental_analyzer, legacy);
writeChar(')', out);
break;
case ActionsDAG::ActionType::FUNCTION:
@ -75,17 +75,17 @@ void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & o
writeCString(", ", out);
first = false;
appendColumnNameWithoutAlias(*arg, out, enable_analyzer, legacy);
appendColumnNameWithoutAlias(*arg, out, allow_experimental_analyzer, legacy);
}
writeChar(')', out);
}
}
}
String getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool enable_analyzer, bool legacy = false)
String getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool allow_experimental_analyzer, bool legacy = false)
{
WriteBufferFromOwnString out;
appendColumnNameWithoutAlias(node, out, enable_analyzer, legacy);
appendColumnNameWithoutAlias(node, out, allow_experimental_analyzer, legacy);
return std::move(out.str());
}
@ -131,7 +131,7 @@ std::string RPNBuilderTreeNode::getColumnName() const
if (ast_node)
return ast_node->getColumnNameWithoutAlias();
else
return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().enable_analyzer);
return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().allow_experimental_analyzer);
}
std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const
@ -144,7 +144,7 @@ std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const
}
else
{
return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().enable_analyzer, true /*legacy*/);
return getColumnNameWithoutAlias(*dag_node, getTreeContext().getSettings().allow_experimental_analyzer, true /*legacy*/);
}
}

View File

@ -397,7 +397,7 @@ void StorageBuffer::read(
/// TODO: Find a way to support projections for StorageBuffer
if (processed_stage > QueryProcessingStage::FetchColumns)
{
if (local_context->getSettingsRef().enable_analyzer)
if (local_context->getSettingsRef().allow_experimental_analyzer)
{
auto storage = std::make_shared<StorageValues>(
getStorageID(),

View File

@ -833,7 +833,7 @@ void StorageDistributed::read(
const auto & settings = local_context->getSettingsRef();
if (settings.enable_analyzer)
if (settings.allow_experimental_analyzer)
{
StorageID remote_storage_id = StorageID::createEmpty();
if (!remote_table_function_ptr)
@ -1057,7 +1057,7 @@ static std::optional<ActionsDAG> getFilterFromQuery(const ASTPtr & ast, ContextP
QueryPlan plan;
SelectQueryOptions options;
options.only_analyze = true;
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
{
InterpreterSelectQueryAnalyzer interpreter(ast, context, options);
plan = std::move(interpreter).extractQueryPlan();
@ -1611,7 +1611,7 @@ ClusterPtr StorageDistributed::skipUnusedShards(
const StorageSnapshotPtr & storage_snapshot,
ContextPtr local_context) const
{
if (local_context->getSettingsRef().enable_analyzer)
if (local_context->getSettingsRef().allow_experimental_analyzer)
return skipUnusedShardsWithAnalyzer(cluster, query_info, storage_snapshot, local_context);
const auto & select = query_info.query->as<ASTSelectQuery &>();

View File

@ -150,7 +150,7 @@ void StorageExecutable::read(
for (auto & input_query : input_queries)
{
QueryPipelineBuilder builder;
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
builder = InterpreterSelectQueryAnalyzer(input_query, context, {}).buildQueryPipeline();
else
builder = InterpreterSelectWithUnionQuery(input_query, context, {}).buildQueryPipeline();

View File

@ -590,7 +590,7 @@ std::vector<ReadFromMerge::ChildPlan> ReadFromMerge::createChildrenPlans(SelectQ
auto modified_query_info
= getModifiedQueryInfo(modified_context, table, nested_storage_snaphsot, real_column_names, column_names_as_aliases, aliases);
if (!context->getSettingsRef().enable_analyzer)
if (!context->getSettingsRef().allow_experimental_analyzer)
{
auto storage_columns = storage_metadata_snapshot->getColumns();
auto syntax_result = TreeRewriter(context).analyzeSelect(
@ -1047,13 +1047,13 @@ void ReadFromMerge::addVirtualColumns(
const StorageWithLockAndName & storage_with_lock) const
{
const auto & [database_name, _, storage, table_name] = storage_with_lock;
bool enable_analyzer = context->getSettingsRef().enable_analyzer;
bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer;
/// Add virtual columns if we don't already have them.
Block plan_header = child.plan.getCurrentDataStream().header;
if (enable_analyzer)
if (allow_experimental_analyzer)
{
String table_alias = modified_query_info.query_tree->as<QueryNode>()->getJoinTree()->as<TableNode>()->getAlias();
@ -1133,8 +1133,8 @@ QueryPipelineBuilderPtr ReadFromMerge::buildPipeline(
if (!builder->initialized())
return builder;
bool enable_analyzer = context->getSettingsRef().enable_analyzer;
if (processed_stage > child.stage || (enable_analyzer && processed_stage != QueryProcessingStage::FetchColumns))
bool allow_experimental_analyzer = context->getSettingsRef().allow_experimental_analyzer;
if (processed_stage > child.stage || (allow_experimental_analyzer && processed_stage != QueryProcessingStage::FetchColumns))
{
/** Materialization is needed, since from distributed storage the constants come materialized.
* If you do not do this, different types (Const and non-Const) columns will be produced in different threads,
@ -1168,7 +1168,7 @@ ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable(
modified_select.setFinal();
}
bool enable_analyzer = modified_context->getSettingsRef().enable_analyzer;
bool allow_experimental_analyzer = modified_context->getSettingsRef().allow_experimental_analyzer;
auto storage_stage = storage->getQueryProcessingStage(modified_context,
processed_stage,
@ -1201,13 +1201,13 @@ ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable(
row_policy_data_opt->addStorageFilter(source_step_with_filter);
}
}
else if (processed_stage > storage_stage || enable_analyzer)
else if (processed_stage > storage_stage || allow_experimental_analyzer)
{
/// Maximum permissible parallelism is streams_num
modified_context->setSetting("max_threads", streams_num);
modified_context->setSetting("max_streams_to_max_threads_ratio", 1);
if (enable_analyzer)
if (allow_experimental_analyzer)
{
/// Converting query to AST because types might be different in the source table.
/// Need to resolve types again.
@ -1479,7 +1479,7 @@ void ReadFromMerge::convertAndFilterSourceStream(
auto storage_sample_block = snapshot->metadata->getSampleBlock();
auto pipe_columns = before_block_header.getNamesAndTypesList();
if (local_context->getSettingsRef().enable_analyzer)
if (local_context->getSettingsRef().allow_experimental_analyzer)
{
for (const auto & alias : aliases)
{
@ -1522,7 +1522,7 @@ void ReadFromMerge::convertAndFilterSourceStream(
ActionsDAG::MatchColumnsMode convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Name;
if (local_context->getSettingsRef().enable_analyzer
if (local_context->getSettingsRef().allow_experimental_analyzer
&& (child.stage != QueryProcessingStage::FetchColumns || dynamic_cast<const StorageDistributed *>(&snapshot->storage) != nullptr))
convert_actions_match_columns_mode = ActionsDAG::MatchColumnsMode::Position;

View File

@ -208,7 +208,7 @@ void StorageMergeTree::read(
const auto & settings = local_context->getSettingsRef();
/// reading step for parallel replicas with new analyzer is built in Planner, so don't do it here
if (local_context->canUseParallelReplicasOnInitiator() && settings.parallel_replicas_for_non_replicated_merge_tree
&& !settings.enable_analyzer)
&& !settings.allow_experimental_analyzer)
{
ClusterProxy::executeQueryWithParallelReplicas(
query_plan, getStorageID(), processed_stage, query_info.query, local_context, query_info.storage_limits);
@ -216,7 +216,7 @@ void StorageMergeTree::read(
}
if (local_context->canUseParallelReplicasCustomKey() && settings.parallel_replicas_for_non_replicated_merge_tree
&& !settings.enable_analyzer && local_context->getClientInfo().distributed_depth == 0)
&& !settings.allow_experimental_analyzer && local_context->getClientInfo().distributed_depth == 0)
{
if (auto cluster = local_context->getClusterForParallelReplicas();
local_context->canUseParallelReplicasCustomKeyForCluster(*cluster))
@ -244,7 +244,7 @@ void StorageMergeTree::read(
const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower()
&& local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree
&& (!local_context->getSettingsRef().enable_analyzer || query_info.current_table_chosen_for_reading_with_parallel_replicas);
&& (!local_context->getSettingsRef().allow_experimental_analyzer || query_info.current_table_chosen_for_reading_with_parallel_replicas);
if (auto plan = reader.read(
column_names,

View File

@ -5480,13 +5480,13 @@ void StorageReplicatedMergeTree::read(
return;
}
/// reading step for parallel replicas with new analyzer is built in Planner, so don't do it here
if (local_context->canUseParallelReplicasOnInitiator() && !settings.enable_analyzer)
if (local_context->canUseParallelReplicasOnInitiator() && !settings.allow_experimental_analyzer)
{
readParallelReplicasImpl(query_plan, column_names, query_info, local_context, processed_stage);
return;
}
if (local_context->canUseParallelReplicasCustomKey() && !settings.enable_analyzer
if (local_context->canUseParallelReplicasCustomKey() && !settings.allow_experimental_analyzer
&& local_context->getClientInfo().distributed_depth == 0)
{
if (auto cluster = local_context->getClusterForParallelReplicas();
@ -5555,7 +5555,7 @@ void StorageReplicatedMergeTree::readLocalImpl(
const size_t num_streams)
{
const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower()
&& (!local_context->getSettingsRef().enable_analyzer
&& (!local_context->getSettingsRef().allow_experimental_analyzer
|| query_info.current_table_chosen_for_reading_with_parallel_replicas);
auto plan = reader.read(

View File

@ -164,7 +164,7 @@ void StorageView::read(
auto options = SelectQueryOptions(QueryProcessingStage::Complete, 0, false, query_info.settings_limit_offset_done);
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
{
InterpreterSelectQueryAnalyzer interpreter(current_inner_query, getViewContext(context, storage_snapshot), options, column_names);
interpreter.addStorageLimits(*query_info.storage_limits);

View File

@ -172,7 +172,7 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType
/// with subqueries it's possible that new analyzer will be enabled in ::read method
/// of underlying storage when all other parts of infra are not ready for it
/// (built with old analyzer).
context_copy->setSetting("enable_analyzer", false);
context_copy->setSetting("allow_experimental_analyzer", false);
auto syntax_analyzer_result = TreeRewriter(context_copy).analyze(ast, columns);
ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context_copy);
auto dag = analyzer.getActionsDAG(false);

View File

@ -1197,7 +1197,7 @@ StorageWindowView::StorageWindowView(
, fire_signal_timeout_s(context_->getSettingsRef().wait_for_window_view_fire_signal_timeout.totalSeconds())
, clean_interval_usec(context_->getSettingsRef().window_view_clean_interval.totalMicroseconds())
{
if (context_->getSettingsRef().enable_analyzer)
if (context_->getSettingsRef().allow_experimental_analyzer)
disabled_due_to_analyzer = true;
if (mode <= LoadingStrictnessLevel::CREATE)
@ -1753,9 +1753,9 @@ StoragePtr StorageWindowView::getTargetTable() const
void StorageWindowView::throwIfWindowViewIsDisabled(ContextPtr local_context) const
{
if (disabled_due_to_analyzer || (local_context && local_context->getSettingsRef().enable_analyzer))
if (disabled_due_to_analyzer || (local_context && local_context->getSettingsRef().allow_experimental_analyzer))
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Experimental WINDOW VIEW feature is not supported "
"in the current infrastructure for query analysis (the setting 'enable_analyzer')");
"in the current infrastructure for query analysis (the setting 'allow_experimental_analyzer')");
}
void registerStorageWindowView(StorageFactory & factory)

View File

@ -50,7 +50,7 @@ ColumnsDescription TableFunctionView::getActualTableStructure(ContextPtr context
Block sample_block;
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.children[0], context);
else
sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(create.children[0], context);

View File

@ -114,7 +114,7 @@ bool TableFunctionViewIfPermitted::isPermitted(const ContextPtr & context, const
try
{
if (context->getSettingsRef().enable_analyzer)
if (context->getSettingsRef().allow_experimental_analyzer)
{
sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.children[0], context);
}