This commit is contained in:
Nikolai Kochetov 2023-08-23 18:51:21 +00:00
parent 33b8b93d1b
commit cb851fcee0
3 changed files with 5 additions and 89 deletions

View File

@ -2,17 +2,20 @@
#include <Columns/ColumnConst.h>
#include <Columns/ColumnSet.h>
#include <Columns/ColumnSet.h>
#include <Common/typeid_cast.h>
#include <Core/Block.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context.h>
#include <Interpreters/castColumn.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/FunctionNameNormalizer.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/TreeRewriter.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
@ -21,11 +24,6 @@
#include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/typeid_cast.h>
#include <Interpreters/FunctionNameNormalizer.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h>
#include <Functions/IFunction.h>
#include <unordered_map>
@ -396,7 +394,7 @@ namespace
{
DisjunctionList res;
for (const auto & lhs_map : lhs)
for(const auto & rhs_map : rhs)
for (const auto & rhs_map : rhs)
if (auto conj = andConjunctions(lhs_map, rhs_map))
res.emplace_back(std::move(*conj));
@ -558,24 +556,6 @@ namespace
return res;
}
// bool isIndependentSubtree(const ActionsDAG::Node * node, const MatchedTrees::Matches & matches)
// {
// std::stack<const ActionsDAG::Node *> stack;
// stack.push(node);
// while (!stack.empty())
// {
// const auto * cur = stack.top();
// stack.pop();
// if (findMatch(cur, matches))
// return false;
// for (const auto * child : node->children)
// stack.push(child);
// }
// return true;
// }
std::optional<DisjunctionList> analyze(const ActionsDAG::Node * node, const MatchedTrees::Matches & matches, const ContextPtr & context, size_t max_elements)
{
if (node->type == ActionsDAG::ActionType::FUNCTION)

View File

@ -409,63 +409,6 @@ StorageDistributed::StorageDistributed(
{
}
// QueryProcessingStage::Enum
// StorageDistributed::getQueryProcessingStageAnalyzer(
// [[maybe_unused]] ContextPtr local_context,
// [[maybe_unused]] QueryProcessingStage::Enum to_stage,
// [[maybe_unused]] const StorageSnapshotPtr & storage_snapshot,
// [[maybe_unused]] SelectQueryInfo & query_info) const
// {
// if (query_info.query_tree)
// LOG_INFO(log, "getQueryProcessingStageAnalyzer tree {}", query_info.query_tree->dumpTree());
// if (query_info.table_expression)
// LOG_INFO(log, "getQueryProcessingStageAnalyzer table_expression {}", query_info.table_expression->dumpTree());
// if (query_info.filter_actions_dag)
// LOG_INFO(log, "getQueryProcessingStageAnalyzer dag {}", query_info.filter_actions_dag->dumpDAG());
// const auto & settings = local_context->getSettingsRef();
// ClusterPtr cluster = getCluster();
// size_t nodes = getClusterQueriedNodes(settings, cluster);
// if (query_info.use_custom_key)
// {
// LOG_INFO(log, "Single shard cluster used with custom_key, transforming replicas into virtual shards");
// query_info.cluster = cluster->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas);
// }
// else
// {
// query_info.cluster = cluster;
// if (nodes > 1 && settings.optimize_skip_unused_shards)
// {
// /// Always calculate optimized cluster here, to avoid conditions during read()
// /// (Anyway it will be calculated in the read())
// ClusterPtr optimized_cluster = getOptimizedCluster(local_context, storage_snapshot, query_info);
// if (optimized_cluster)
// {
// LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}",
// makeFormattedListOfShards(optimized_cluster));
// cluster = optimized_cluster;
// query_info.optimized_cluster = cluster;
// nodes = getClusterQueriedNodes(settings, cluster);
// }
// else
// {
// LOG_DEBUG(log, "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster{}",
// has_sharding_key ? "" : " (no sharding key)");
// }
// }
// }
// query_info.cluster = cluster;
// return QueryProcessingStage::WithMergeableState;
// }
QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(
ContextPtr local_context,
QueryProcessingStage::Enum to_stage,
@ -473,10 +416,6 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(
SelectQueryInfo & query_info) const
{
const auto & settings = local_context->getSettingsRef();
// if (settings.allow_experimental_analyzer)
// return getQueryProcessingStageAnalyzer(local_context, to_stage, storage_snapshot, query_info);
ClusterPtr cluster = getCluster();
size_t nodes = getClusterQueriedNodes(settings, cluster);

View File

@ -209,9 +209,6 @@ private:
std::optional<QueryProcessingStage::Enum> getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, const Settings & settings) const;
std::optional<QueryProcessingStage::Enum> getOptimizedQueryProcessingStageAnalyzer(const SelectQueryInfo & query_info, const Settings & settings) const;
// QueryProcessingStage::Enum
// getQueryProcessingStageAnalyzer(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const;
size_t getRandomShardIndex(const Cluster::ShardsInfo & shards);
std::string getClusterName() const { return cluster_name.empty() ? "<remote>" : cluster_name; }