mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge pull request #43321 from kitaisreal/analyzer-small-fixes
Analyzer small fixes
This commit is contained in:
commit
43a451d55e
60
src/Analyzer/HashUtils.h
Normal file
60
src/Analyzer/HashUtils.h
Normal file
@ -0,0 +1,60 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Analyzer/IQueryTreeNode.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/** This structure holds query tree node ptr and its hash. It can be used as hash map key to avoid unnecessary hash
|
||||||
|
* recalculations.
|
||||||
|
*
|
||||||
|
* Example of usage:
|
||||||
|
* std::unordered_map<QueryTreeNodeConstRawPtrWithHash, std::string> map;
|
||||||
|
*/
|
||||||
|
template <typename QueryTreeNodePtrType>
|
||||||
|
struct QueryTreeNodeWithHash
|
||||||
|
{
|
||||||
|
QueryTreeNodeWithHash(QueryTreeNodePtrType node_) /// NOLINT
|
||||||
|
: node(std::move(node_))
|
||||||
|
, hash(node->getTreeHash().first)
|
||||||
|
{}
|
||||||
|
|
||||||
|
QueryTreeNodePtrType node = nullptr;
|
||||||
|
size_t hash = 0;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
inline bool operator==(const QueryTreeNodeWithHash<T> & lhs, const QueryTreeNodeWithHash<T> & rhs)
|
||||||
|
{
|
||||||
|
return lhs.hash == rhs.hash && lhs.node->isEqual(*rhs.node);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
inline bool operator!=(const QueryTreeNodeWithHash<T> & lhs, const QueryTreeNodeWithHash<T> & rhs)
|
||||||
|
{
|
||||||
|
return !(lhs == rhs);
|
||||||
|
}
|
||||||
|
|
||||||
|
using QueryTreeNodePtrWithHash = QueryTreeNodeWithHash<QueryTreeNodePtr>;
|
||||||
|
using QueryTreeNodeRawPtrWithHash = QueryTreeNodeWithHash<IQueryTreeNode *>;
|
||||||
|
using QueryTreeNodeConstRawPtrWithHash = QueryTreeNodeWithHash<const IQueryTreeNode *>;
|
||||||
|
|
||||||
|
using QueryTreeNodePtrWithHashSet = std::unordered_set<QueryTreeNodePtrWithHash>;
|
||||||
|
using QueryTreeNodeConstRawPtrWithHashSet = std::unordered_set<QueryTreeNodeConstRawPtrWithHash>;
|
||||||
|
|
||||||
|
template <typename Value>
|
||||||
|
using QueryTreeNodePtrWithHashMap = std::unordered_map<QueryTreeNodePtrWithHash, Value>;
|
||||||
|
|
||||||
|
template <typename Value>
|
||||||
|
using QueryTreeNodeConstRawPtrWithHashMap = std::unordered_map<QueryTreeNodeConstRawPtrWithHash, Value>;
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
struct std::hash<DB::QueryTreeNodeWithHash<T>>
|
||||||
|
{
|
||||||
|
size_t operator()(const DB::QueryTreeNodeWithHash<T> & node_with_hash) const
|
||||||
|
{
|
||||||
|
return node_with_hash.hash;
|
||||||
|
}
|
||||||
|
};
|
@ -8,6 +8,7 @@
|
|||||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||||
#include <Analyzer/FunctionNode.h>
|
#include <Analyzer/FunctionNode.h>
|
||||||
#include <Analyzer/ConstantNode.h>
|
#include <Analyzer/ConstantNode.h>
|
||||||
|
#include <Analyzer/HashUtils.h>
|
||||||
|
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
@ -48,43 +49,24 @@ public:
|
|||||||
/// Do not apply for `count()` with without arguments or `count(*)`, only `count(x)` is supported.
|
/// Do not apply for `count()` with without arguments or `count(*)`, only `count(x)` is supported.
|
||||||
return;
|
return;
|
||||||
|
|
||||||
mapping[QueryTreeNodeWithHash(argument_nodes[0])].push_back(&node);
|
argument_to_functions_mapping[argument_nodes[0]].push_back(&node);
|
||||||
}
|
}
|
||||||
|
|
||||||
struct QueryTreeNodeWithHash
|
|
||||||
{
|
|
||||||
const QueryTreeNodePtr & node;
|
|
||||||
IQueryTreeNode::Hash hash;
|
|
||||||
|
|
||||||
explicit QueryTreeNodeWithHash(const QueryTreeNodePtr & node_)
|
|
||||||
: node(node_)
|
|
||||||
, hash(node->getTreeHash())
|
|
||||||
{}
|
|
||||||
|
|
||||||
bool operator==(const QueryTreeNodeWithHash & rhs) const
|
|
||||||
{
|
|
||||||
return hash == rhs.hash && node->isEqual(*rhs.node);
|
|
||||||
}
|
|
||||||
|
|
||||||
struct Hash
|
|
||||||
{
|
|
||||||
size_t operator() (const QueryTreeNodeWithHash & key) const { return key.hash.first ^ key.hash.second; }
|
|
||||||
};
|
|
||||||
};
|
|
||||||
|
|
||||||
/// argument -> list of sum/count/avg functions with this argument
|
/// argument -> list of sum/count/avg functions with this argument
|
||||||
std::unordered_map<QueryTreeNodeWithHash, std::vector<QueryTreeNodePtr *>, QueryTreeNodeWithHash::Hash> mapping;
|
QueryTreeNodePtrWithHashMap<std::vector<QueryTreeNodePtr *>> argument_to_functions_mapping;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
std::unordered_set<String> names_to_collect;
|
std::unordered_set<String> names_to_collect;
|
||||||
};
|
};
|
||||||
|
|
||||||
QueryTreeNodePtr createResolvedFunction(ContextPtr context, const String & name, DataTypePtr result_type, QueryTreeNodes arguments)
|
QueryTreeNodePtr createResolvedFunction(const ContextPtr & context, const String & name, const DataTypePtr & result_type, QueryTreeNodes arguments)
|
||||||
{
|
{
|
||||||
auto function_node = std::make_shared<FunctionNode>(name);
|
auto function_node = std::make_shared<FunctionNode>(name);
|
||||||
|
|
||||||
auto function = FunctionFactory::instance().get(name, context);
|
auto function = FunctionFactory::instance().get(name, context);
|
||||||
function_node->resolveAsFunction(std::move(function), result_type);
|
function_node->resolveAsFunction(std::move(function), result_type);
|
||||||
function_node->getArguments().getNodes() = std::move(arguments);
|
function_node->getArguments().getNodes() = std::move(arguments);
|
||||||
|
|
||||||
return function_node;
|
return function_node;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -94,21 +76,20 @@ FunctionNodePtr createResolvedAggregateFunction(const String & name, const Query
|
|||||||
|
|
||||||
AggregateFunctionProperties properties;
|
AggregateFunctionProperties properties;
|
||||||
auto aggregate_function = AggregateFunctionFactory::instance().get(name, {argument->getResultType()}, parameters, properties);
|
auto aggregate_function = AggregateFunctionFactory::instance().get(name, {argument->getResultType()}, parameters, properties);
|
||||||
|
|
||||||
function_node->resolveAsAggregateFunction(aggregate_function, aggregate_function->getReturnType());
|
function_node->resolveAsAggregateFunction(aggregate_function, aggregate_function->getReturnType());
|
||||||
|
function_node->getArguments().getNodes() = { argument };
|
||||||
|
|
||||||
function_node->getArgumentsNode() = std::make_shared<ListNode>(QueryTreeNodes{argument});
|
|
||||||
return function_node;
|
return function_node;
|
||||||
}
|
}
|
||||||
|
|
||||||
QueryTreeNodePtr createTupleElementFunction(ContextPtr context, DataTypePtr result_type, QueryTreeNodePtr argument, UInt64 index)
|
QueryTreeNodePtr createTupleElementFunction(const ContextPtr & context, const DataTypePtr & result_type, QueryTreeNodePtr argument, UInt64 index)
|
||||||
{
|
{
|
||||||
return createResolvedFunction(context, "tupleElement", result_type, {argument, std::make_shared<ConstantNode>(index)});
|
return createResolvedFunction(context, "tupleElement", result_type, {std::move(argument), std::make_shared<ConstantNode>(index)});
|
||||||
}
|
}
|
||||||
|
|
||||||
QueryTreeNodePtr createArrayElementFunction(ContextPtr context, DataTypePtr result_type, QueryTreeNodePtr argument, UInt64 index)
|
QueryTreeNodePtr createArrayElementFunction(const ContextPtr & context, const DataTypePtr & result_type, QueryTreeNodePtr argument, UInt64 index)
|
||||||
{
|
{
|
||||||
return createResolvedFunction(context, "arrayElement", result_type, {argument, std::make_shared<ConstantNode>(index)});
|
return createResolvedFunction(context, "arrayElement", result_type, {std::move(argument), std::make_shared<ConstantNode>(index)});
|
||||||
}
|
}
|
||||||
|
|
||||||
void replaceWithSumCount(QueryTreeNodePtr & node, const FunctionNodePtr & sum_count_node, ContextPtr context)
|
void replaceWithSumCount(QueryTreeNodePtr & node, const FunctionNodePtr & sum_count_node, ContextPtr context)
|
||||||
@ -151,6 +132,7 @@ FunctionNodePtr createFusedQuantilesNode(const std::vector<QueryTreeNodePtr *> n
|
|||||||
{
|
{
|
||||||
Array parameters;
|
Array parameters;
|
||||||
parameters.reserve(nodes.size());
|
parameters.reserve(nodes.size());
|
||||||
|
|
||||||
for (const auto * node : nodes)
|
for (const auto * node : nodes)
|
||||||
{
|
{
|
||||||
const FunctionNode & function_node = (*node)->as<const FunctionNode &>();
|
const FunctionNode & function_node = (*node)->as<const FunctionNode &>();
|
||||||
@ -172,6 +154,7 @@ FunctionNodePtr createFusedQuantilesNode(const std::vector<QueryTreeNodePtr *> n
|
|||||||
|
|
||||||
parameters.push_back(constant_value->getValue());
|
parameters.push_back(constant_value->getValue());
|
||||||
}
|
}
|
||||||
|
|
||||||
return createResolvedAggregateFunction("quantiles", argument, parameters);
|
return createResolvedAggregateFunction("quantiles", argument, parameters);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -181,7 +164,7 @@ void tryFuseSumCountAvg(QueryTreeNodePtr query_tree_node, ContextPtr context)
|
|||||||
FuseFunctionsVisitor visitor({"sum", "count", "avg"});
|
FuseFunctionsVisitor visitor({"sum", "count", "avg"});
|
||||||
visitor.visit(query_tree_node);
|
visitor.visit(query_tree_node);
|
||||||
|
|
||||||
for (auto & [argument, nodes] : visitor.mapping)
|
for (auto & [argument, nodes] : visitor.argument_to_functions_mapping)
|
||||||
{
|
{
|
||||||
if (nodes.size() < 2)
|
if (nodes.size() < 2)
|
||||||
continue;
|
continue;
|
||||||
@ -199,24 +182,22 @@ void tryFuseQuantiles(QueryTreeNodePtr query_tree_node, ContextPtr context)
|
|||||||
{
|
{
|
||||||
FuseFunctionsVisitor visitor_quantile({"quantile"});
|
FuseFunctionsVisitor visitor_quantile({"quantile"});
|
||||||
visitor_quantile.visit(query_tree_node);
|
visitor_quantile.visit(query_tree_node);
|
||||||
for (auto & [argument, nodes] : visitor_quantile.mapping)
|
|
||||||
|
for (auto & [argument, nodes] : visitor_quantile.argument_to_functions_mapping)
|
||||||
{
|
{
|
||||||
if (nodes.size() < 2)
|
size_t nodes_size = nodes.size();
|
||||||
|
if (nodes_size < 2)
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
auto quantiles_node = createFusedQuantilesNode(nodes, argument.node);
|
auto quantiles_node = createFusedQuantilesNode(nodes, argument.node);
|
||||||
auto result_array_type = std::dynamic_pointer_cast<const DataTypeArray>(quantiles_node->getResultType());
|
auto result_array_type = std::dynamic_pointer_cast<const DataTypeArray>(quantiles_node->getResultType());
|
||||||
if (!result_array_type)
|
if (!result_array_type)
|
||||||
{
|
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||||
"Unexpected return type '{}' of function '{}', should be array",
|
"Unexpected return type '{}' of function '{}', should be array",
|
||||||
quantiles_node->getResultType(), quantiles_node->getFunctionName());
|
quantiles_node->getResultType(), quantiles_node->getFunctionName());
|
||||||
}
|
|
||||||
|
|
||||||
for (size_t i = 0; i < nodes.size(); ++i)
|
for (size_t i = 0; i < nodes_size; ++i)
|
||||||
{
|
|
||||||
*nodes[i] = createArrayElementFunction(context, result_array_type->getNestedType(), quantiles_node, i + 1);
|
*nodes[i] = createArrayElementFunction(context, result_array_type->getNestedType(), quantiles_node, i + 1);
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||||
#include <Analyzer/QueryNode.h>
|
#include <Analyzer/QueryNode.h>
|
||||||
#include <Analyzer/SortNode.h>
|
#include <Analyzer/SortNode.h>
|
||||||
|
#include <Analyzer/HashUtils.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -10,35 +11,6 @@ namespace DB
|
|||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
struct QueryTreeNodeWithHash
|
|
||||||
{
|
|
||||||
explicit QueryTreeNodeWithHash(const IQueryTreeNode * node_)
|
|
||||||
: node(node_)
|
|
||||||
, hash(node->getTreeHash().first)
|
|
||||||
{}
|
|
||||||
|
|
||||||
const IQueryTreeNode * node = nullptr;
|
|
||||||
size_t hash = 0;
|
|
||||||
};
|
|
||||||
|
|
||||||
struct QueryTreeNodeWithHashHash
|
|
||||||
{
|
|
||||||
size_t operator()(const QueryTreeNodeWithHash & node_with_hash) const
|
|
||||||
{
|
|
||||||
return node_with_hash.hash;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
struct QueryTreeNodeWithHashEqualTo
|
|
||||||
{
|
|
||||||
bool operator()(const QueryTreeNodeWithHash & lhs_node, const QueryTreeNodeWithHash & rhs_node) const
|
|
||||||
{
|
|
||||||
return lhs_node.hash == rhs_node.hash && lhs_node.node->isEqual(*rhs_node.node);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
using QueryTreeNodeWithHashSet = std::unordered_set<QueryTreeNodeWithHash, QueryTreeNodeWithHashHash, QueryTreeNodeWithHashEqualTo>;
|
|
||||||
|
|
||||||
class OrderByLimitByDuplicateEliminationVisitor : public InDepthQueryTreeVisitor<OrderByLimitByDuplicateEliminationVisitor>
|
class OrderByLimitByDuplicateEliminationVisitor : public InDepthQueryTreeVisitor<OrderByLimitByDuplicateEliminationVisitor>
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
@ -93,7 +65,7 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
QueryTreeNodeWithHashSet unique_expressions_nodes_set;
|
QueryTreeNodeConstRawPtrWithHashSet unique_expressions_nodes_set;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -519,7 +519,7 @@ public:
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
QueryTreeNodes expressions;
|
QueryTreeNodes expressions;
|
||||||
std::unordered_map<std::string, std::vector<QueryTreeNodePtr>> alias_name_to_expressions;
|
std::unordered_map<std::string, QueryTreeNodes> alias_name_to_expressions;
|
||||||
};
|
};
|
||||||
|
|
||||||
/** Projection names is name of query tree node that is used in projection part of query node.
|
/** Projection names is name of query tree node that is used in projection part of query node.
|
||||||
@ -2239,18 +2239,19 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier
|
|||||||
auto & alias_identifier_node = it->second->as<IdentifierNode &>();
|
auto & alias_identifier_node = it->second->as<IdentifierNode &>();
|
||||||
auto identifier = alias_identifier_node.getIdentifier();
|
auto identifier = alias_identifier_node.getIdentifier();
|
||||||
auto lookup_result = tryResolveIdentifier(IdentifierLookup{identifier, identifier_lookup.lookup_context}, scope, identifier_resolve_settings);
|
auto lookup_result = tryResolveIdentifier(IdentifierLookup{identifier, identifier_lookup.lookup_context}, scope, identifier_resolve_settings);
|
||||||
if (!lookup_result.isResolved())
|
if (!lookup_result.resolved_identifier)
|
||||||
{
|
{
|
||||||
std::unordered_set<Identifier> valid_identifiers;
|
std::unordered_set<Identifier> valid_identifiers;
|
||||||
collectScopeWithParentScopesValidIdentifiersForTypoCorrection(identifier, scope, true, false, false, valid_identifiers);
|
collectScopeWithParentScopesValidIdentifiersForTypoCorrection(identifier, scope, true, false, false, valid_identifiers);
|
||||||
|
|
||||||
auto hints = collectIdentifierTypoHints(identifier, valid_identifiers);
|
auto hints = collectIdentifierTypoHints(identifier, valid_identifiers);
|
||||||
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {} identifier '{}' in scope {}{}",
|
|
||||||
toStringLowercase(IdentifierLookupContext::EXPRESSION),
|
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {} identifier '{}'. In scope {}{}",
|
||||||
|
toStringLowercase(identifier_lookup.lookup_context),
|
||||||
identifier.getFullName(),
|
identifier.getFullName(),
|
||||||
scope.scope_node->formatASTForErrorMessage(),
|
scope.scope_node->formatASTForErrorMessage(),
|
||||||
getHintsErrorMessageSuffix(hints));
|
getHintsErrorMessageSuffix(hints));
|
||||||
}
|
}
|
||||||
|
|
||||||
it->second = lookup_result.resolved_identifier;
|
it->second = lookup_result.resolved_identifier;
|
||||||
|
|
||||||
/** During collection of aliases if node is identifier and has alias, we cannot say if it is
|
/** During collection of aliases if node is identifier and has alias, we cannot say if it is
|
||||||
@ -2261,9 +2262,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier
|
|||||||
* If we resolved identifier node as function, we must remove identifier node alias from
|
* If we resolved identifier node as function, we must remove identifier node alias from
|
||||||
* expression alias map.
|
* expression alias map.
|
||||||
*/
|
*/
|
||||||
if (identifier_lookup.isExpressionLookup() && it->second)
|
if (identifier_lookup.isExpressionLookup())
|
||||||
scope.alias_name_to_lambda_node.erase(identifier_bind_part);
|
scope.alias_name_to_lambda_node.erase(identifier_bind_part);
|
||||||
else if (identifier_lookup.isFunctionLookup() && it->second)
|
else if (identifier_lookup.isFunctionLookup())
|
||||||
scope.alias_name_to_expression_node.erase(identifier_bind_part);
|
scope.alias_name_to_expression_node.erase(identifier_bind_part);
|
||||||
|
|
||||||
scope.expressions_in_resolve_process_stack.popNode();
|
scope.expressions_in_resolve_process_stack.popNode();
|
||||||
@ -3271,11 +3272,9 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(
|
|||||||
|
|
||||||
if (auto * array_join_node = table_expression->as<ArrayJoinNode>())
|
if (auto * array_join_node = table_expression->as<ArrayJoinNode>())
|
||||||
{
|
{
|
||||||
size_t table_expressions_column_nodes_with_names_stack_size = table_expressions_column_nodes_with_names_stack.size();
|
if (table_expressions_column_nodes_with_names_stack.empty())
|
||||||
if (table_expressions_column_nodes_with_names_stack_size < 1)
|
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||||
"Expected at least 1 table expressions on stack before ARRAY JOIN processing. Actual {}",
|
"Expected at least 1 table expressions on stack before ARRAY JOIN processing");
|
||||||
table_expressions_column_nodes_with_names_stack_size);
|
|
||||||
|
|
||||||
auto & table_expression_column_nodes_with_names = table_expressions_column_nodes_with_names_stack.back();
|
auto & table_expression_column_nodes_with_names = table_expressions_column_nodes_with_names_stack.back();
|
||||||
|
|
||||||
|
@ -87,8 +87,8 @@ void JoinClause::dump(WriteBuffer & buffer) const
|
|||||||
{
|
{
|
||||||
const auto & asof_condition = asof_conditions[i];
|
const auto & asof_condition = asof_conditions[i];
|
||||||
|
|
||||||
buffer << "key_index: " << asof_condition.key_index;
|
buffer << " key_index: " << asof_condition.key_index;
|
||||||
buffer << "inequality: " << toString(asof_condition.asof_inequality);
|
buffer << " inequality: " << toString(asof_condition.asof_inequality);
|
||||||
|
|
||||||
if (i + 1 != asof_conditions_size)
|
if (i + 1 != asof_conditions_size)
|
||||||
buffer << ',';
|
buffer << ',';
|
||||||
|
@ -183,19 +183,19 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
/// Valid for table, table function, query, union, array join table expression nodes
|
/// Valid for table, table function, array join, query, union nodes
|
||||||
NamesAndTypesList columns;
|
NamesAndTypesList columns;
|
||||||
|
|
||||||
/// Valid for table, table function, query, union, array join table expression nodes
|
/// Valid for table, table function, array join, query, union nodes
|
||||||
NameSet columns_names;
|
NameSet columns_names;
|
||||||
|
|
||||||
/// Valid only for table table expression node
|
/// Valid only for table node
|
||||||
NameSet alias_columns_names;
|
NameSet alias_columns_names;
|
||||||
|
|
||||||
/// Valid for table, table function, query, union table, array join expression nodes
|
/// Valid for table, table function, array join, query, union nodes
|
||||||
ColumnNameToColumnIdentifier column_name_to_column_identifier;
|
ColumnNameToColumnIdentifier column_name_to_column_identifier;
|
||||||
|
|
||||||
/// Valid for table, table function, query, union table, array join expression nodes
|
/// Valid for table, table function, array join, query, union nodes
|
||||||
ColumnIdentifierToColumnName column_identifier_to_column_name;
|
ColumnIdentifierToColumnName column_identifier_to_column_name;
|
||||||
|
|
||||||
/// Is storage remote
|
/// Is storage remote
|
||||||
|
@ -0,0 +1 @@
|
|||||||
|
String Value_1
|
@ -0,0 +1,36 @@
|
|||||||
|
SET allow_experimental_analyzer = 1;
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS test_table_join_1;
|
||||||
|
CREATE TABLE test_table_join_1
|
||||||
|
(
|
||||||
|
id UInt8,
|
||||||
|
value String
|
||||||
|
)
|
||||||
|
ENGINE = TinyLog;
|
||||||
|
|
||||||
|
INSERT INTO test_table_join_1 VALUES (0, 'Value_0');
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS test_table_join_2;
|
||||||
|
CREATE TABLE test_table_join_2
|
||||||
|
(
|
||||||
|
id UInt16,
|
||||||
|
value String
|
||||||
|
)
|
||||||
|
ENGINE = TinyLog;
|
||||||
|
|
||||||
|
INSERT INTO test_table_join_2 VALUES (0, 'Value_1');
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
toTypeName(t2_value),
|
||||||
|
t2.value AS t2_value
|
||||||
|
FROM test_table_join_1 AS t1
|
||||||
|
INNER JOIN test_table_join_2 USING (id); -- { serverError 47 };
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
toTypeName(t2_value),
|
||||||
|
t2.value AS t2_value
|
||||||
|
FROM test_table_join_1 AS t1
|
||||||
|
INNER JOIN test_table_join_2 AS t2 USING (id);
|
||||||
|
|
||||||
|
DROP TABLE test_table_join_1;
|
||||||
|
DROP TABLE test_table_join_2;
|
Loading…
Reference in New Issue
Block a user