mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Merge branch 'ClickHouse:master' into fix-sort
This commit is contained in:
commit
c4e17f87e7
@ -7,6 +7,7 @@
|
||||
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/QueryNode.h>
|
||||
#include <Analyzer/TableFunctionNode.h>
|
||||
#include <Analyzer/UnionNode.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
@ -90,26 +91,25 @@ private:
|
||||
template <typename Derived>
|
||||
using ConstInDepthQueryTreeVisitor = InDepthQueryTreeVisitor<Derived, true /*const_visitor*/>;
|
||||
|
||||
/** Same as InDepthQueryTreeVisitor and additionally keeps track of current scope context.
|
||||
/** Same as InDepthQueryTreeVisitor (but has a different interface) and additionally keeps track of current scope context.
|
||||
* This can be useful if your visitor has special logic that depends on current scope context.
|
||||
*
|
||||
* To specify behavior of the visitor you can implement following methods in derived class:
|
||||
* 1. needChildVisit – This methods allows to skip subtree.
|
||||
* 2. enterImpl – This method is called before children are processed.
|
||||
* 3. leaveImpl – This method is called after children are processed.
|
||||
*/
|
||||
template <typename Derived, bool const_visitor = false>
|
||||
class InDepthQueryTreeVisitorWithContext
|
||||
{
|
||||
public:
|
||||
using VisitQueryTreeNodeType = std::conditional_t<const_visitor, const QueryTreeNodePtr, QueryTreeNodePtr>;
|
||||
using VisitQueryTreeNodeType = QueryTreeNodePtr;
|
||||
|
||||
explicit InDepthQueryTreeVisitorWithContext(ContextPtr context, size_t initial_subquery_depth = 0)
|
||||
: current_context(std::move(context))
|
||||
, subquery_depth(initial_subquery_depth)
|
||||
{}
|
||||
|
||||
/// Return true if visitor should traverse tree top to bottom, false otherwise
|
||||
bool shouldTraverseTopToBottom() const
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
/// Return true if visitor should visit child, false otherwise
|
||||
bool needChildVisit(VisitQueryTreeNodeType & parent [[maybe_unused]], VisitQueryTreeNodeType & child [[maybe_unused]])
|
||||
{
|
||||
@ -146,18 +146,16 @@ public:
|
||||
|
||||
++subquery_depth;
|
||||
|
||||
bool traverse_top_to_bottom = getDerived().shouldTraverseTopToBottom();
|
||||
if (!traverse_top_to_bottom)
|
||||
visitChildren(query_tree_node);
|
||||
getDerived().enterImpl(query_tree_node);
|
||||
|
||||
getDerived().visitImpl(query_tree_node);
|
||||
|
||||
if (traverse_top_to_bottom)
|
||||
visitChildren(query_tree_node);
|
||||
|
||||
getDerived().leaveImpl(query_tree_node);
|
||||
}
|
||||
|
||||
void enterImpl(VisitQueryTreeNodeType & node [[maybe_unused]])
|
||||
{}
|
||||
|
||||
void leaveImpl(VisitQueryTreeNodeType & node [[maybe_unused]])
|
||||
{}
|
||||
private:
|
||||
@ -171,17 +169,31 @@ private:
|
||||
return *static_cast<Derived *>(this);
|
||||
}
|
||||
|
||||
bool shouldSkipSubtree(
|
||||
VisitQueryTreeNodeType & parent,
|
||||
VisitQueryTreeNodeType & child,
|
||||
size_t subtree_index)
|
||||
{
|
||||
bool need_visit_child = getDerived().needChildVisit(parent, child);
|
||||
if (!need_visit_child)
|
||||
return true;
|
||||
|
||||
if (auto * table_function_node = parent->as<TableFunctionNode>())
|
||||
{
|
||||
const auto & unresolved_indexes = table_function_node->getUnresolvedArgumentIndexes();
|
||||
return std::find(unresolved_indexes.begin(), unresolved_indexes.end(), subtree_index) != unresolved_indexes.end();
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
void visitChildren(VisitQueryTreeNodeType & expression)
|
||||
{
|
||||
size_t index = 0;
|
||||
for (auto & child : expression->getChildren())
|
||||
{
|
||||
if (!child)
|
||||
continue;
|
||||
|
||||
bool need_visit_child = getDerived().needChildVisit(expression, child);
|
||||
|
||||
if (need_visit_child)
|
||||
if (child && !shouldSkipSubtree(expression, child, index))
|
||||
visit(child);
|
||||
++index;
|
||||
}
|
||||
}
|
||||
|
||||
@ -189,50 +201,4 @@ private:
|
||||
size_t subquery_depth = 0;
|
||||
};
|
||||
|
||||
template <typename Derived>
|
||||
using ConstInDepthQueryTreeVisitorWithContext = InDepthQueryTreeVisitorWithContext<Derived, true /*const_visitor*/>;
|
||||
|
||||
/** Visitor that use another visitor to visit node only if condition for visiting node is true.
|
||||
* For example, your visitor need to visit only query tree nodes or union nodes.
|
||||
*
|
||||
* Condition interface:
|
||||
* struct Condition
|
||||
* {
|
||||
* bool operator()(VisitQueryTreeNodeType & node)
|
||||
* {
|
||||
* return shouldNestedVisitorVisitNode(node);
|
||||
* }
|
||||
* }
|
||||
*/
|
||||
template <typename Visitor, typename Condition, bool const_visitor = false>
|
||||
class InDepthQueryTreeConditionalVisitor : public InDepthQueryTreeVisitor<InDepthQueryTreeConditionalVisitor<Visitor, Condition, const_visitor>, const_visitor>
|
||||
{
|
||||
public:
|
||||
using Base = InDepthQueryTreeVisitor<InDepthQueryTreeConditionalVisitor<Visitor, Condition, const_visitor>, const_visitor>;
|
||||
using VisitQueryTreeNodeType = typename Base::VisitQueryTreeNodeType;
|
||||
|
||||
explicit InDepthQueryTreeConditionalVisitor(Visitor & visitor_, Condition & condition_)
|
||||
: visitor(visitor_)
|
||||
, condition(condition_)
|
||||
{
|
||||
}
|
||||
|
||||
bool shouldTraverseTopToBottom() const
|
||||
{
|
||||
return visitor.shouldTraverseTopToBottom();
|
||||
}
|
||||
|
||||
void visitImpl(VisitQueryTreeNodeType & query_tree_node)
|
||||
{
|
||||
if (condition(query_tree_node))
|
||||
visitor.visit(query_tree_node);
|
||||
}
|
||||
|
||||
Visitor & visitor;
|
||||
Condition & condition;
|
||||
};
|
||||
|
||||
template <typename Visitor, typename Condition>
|
||||
using ConstInDepthQueryTreeConditionalVisitor = InDepthQueryTreeConditionalVisitor<Visitor, Condition, true /*const_visitor*/>;
|
||||
|
||||
}
|
||||
|
@ -51,13 +51,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<AggregateFunctionsArithmericOperationsVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
/// Traverse tree bottom to top
|
||||
static bool shouldTraverseTopToBottom()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void leaveImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_arithmetic_operations_in_aggregate_functions)
|
||||
return;
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<RewriteArrayExistsToHasVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_rewrite_array_exists_to_has)
|
||||
return;
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<AutoFinalOnQueryPassVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().final)
|
||||
return;
|
||||
|
@ -50,7 +50,7 @@ public:
|
||||
&& settings.max_hyperscan_regexp_total_length == 0;
|
||||
}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
auto * function_node = node->as<FunctionNode>();
|
||||
if (!function_node || function_node->getFunctionName() != "or")
|
||||
|
@ -688,7 +688,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<ConvertQueryToCNFVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
auto * query_node = node->as<QueryNode>();
|
||||
if (!query_node)
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<CountDistinctVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().count_distinct_optimization)
|
||||
return;
|
||||
|
@ -193,7 +193,7 @@ public:
|
||||
return true;
|
||||
}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!isEnabled())
|
||||
return;
|
||||
|
@ -29,7 +29,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<FunctionToSubcolumnsVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node) const
|
||||
void enterImpl(QueryTreeNodePtr & node) const
|
||||
{
|
||||
if (!getSettings().optimize_functions_to_subcolumns)
|
||||
return;
|
||||
|
@ -37,7 +37,7 @@ public:
|
||||
, names_to_collect(names_to_collect_)
|
||||
{}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_syntax_fuse_functions)
|
||||
return;
|
||||
|
@ -46,7 +46,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
void visitImpl(const QueryTreeNodePtr & node)
|
||||
void enterImpl(const QueryTreeNodePtr & node)
|
||||
{
|
||||
auto * function_node = node->as<FunctionNode>();
|
||||
if (!function_node || function_node->getFunctionName() != "grouping")
|
||||
|
@ -23,7 +23,7 @@ public:
|
||||
, multi_if_function_ptr(std::move(multi_if_function_ptr_))
|
||||
{}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_if_chain_to_multiif)
|
||||
return;
|
||||
|
@ -113,7 +113,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<ConvertStringsToEnumVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_if_transform_strings_to_enum)
|
||||
return;
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
: Base(std::move(context))
|
||||
{}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
auto * function_node = node->as<FunctionNode>();
|
||||
|
||||
|
@ -21,7 +21,7 @@ public:
|
||||
, if_function_ptr(std::move(if_function_ptr_))
|
||||
{}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_multiif_to_if)
|
||||
return;
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<NormalizeCountVariantsVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_normalize_count_variants)
|
||||
return;
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
return !child->as<FunctionNode>();
|
||||
}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_group_by_function_keys)
|
||||
return;
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
return true;
|
||||
}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_redundant_functions_in_order_by)
|
||||
return;
|
||||
|
@ -6451,7 +6451,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node,
|
||||
table_function_ptr->parseArguments(table_function_ast, scope_context);
|
||||
|
||||
auto table_function_storage = scope_context->getQueryContext()->executeTableFunction(table_function_ast, table_function_ptr);
|
||||
table_function_node_typed.resolve(std::move(table_function_ptr), std::move(table_function_storage), scope_context);
|
||||
table_function_node_typed.resolve(std::move(table_function_ptr), std::move(table_function_storage), scope_context, std::move(skip_analysis_arguments_indexes));
|
||||
}
|
||||
|
||||
/// Resolve array join node in scope
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<RewriteAggregateFunctionWithIfVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_rewrite_aggregate_function_with_if)
|
||||
return;
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<ShardNumColumnToFunctionVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node) const
|
||||
void enterImpl(QueryTreeNodePtr & node) const
|
||||
{
|
||||
auto * column_node = node->as<ColumnNode>();
|
||||
if (!column_node)
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<SumIfToCountIfVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_rewrite_sum_if_to_count_if)
|
||||
return;
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
using Base = InDepthQueryTreeVisitorWithContext<UniqInjectiveFunctionsEliminationVisitor>;
|
||||
using Base::Base;
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (!getSettings().optimize_injective_functions_inside_uniq)
|
||||
return;
|
||||
|
@ -27,12 +27,13 @@ TableFunctionNode::TableFunctionNode(String table_function_name_)
|
||||
children[arguments_child_index] = std::make_shared<ListNode>();
|
||||
}
|
||||
|
||||
void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context)
|
||||
void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context, std::vector<size_t> unresolved_arguments_indexes_)
|
||||
{
|
||||
table_function = std::move(table_function_value);
|
||||
storage = std::move(storage_value);
|
||||
storage_id = storage->getStorageID();
|
||||
storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context);
|
||||
unresolved_arguments_indexes = std::move(unresolved_arguments_indexes_);
|
||||
}
|
||||
|
||||
const StorageID & TableFunctionNode::getStorageID() const
|
||||
@ -132,6 +133,7 @@ QueryTreeNodePtr TableFunctionNode::cloneImpl() const
|
||||
result->storage_snapshot = storage_snapshot;
|
||||
result->table_expression_modifiers = table_expression_modifiers;
|
||||
result->settings_changes = settings_changes;
|
||||
result->unresolved_arguments_indexes = unresolved_arguments_indexes;
|
||||
|
||||
return result;
|
||||
}
|
||||
|
@ -98,7 +98,7 @@ public:
|
||||
}
|
||||
|
||||
/// Resolve table function with table function, storage and context
|
||||
void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context);
|
||||
void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context, std::vector<size_t> unresolved_arguments_indexes_);
|
||||
|
||||
/// Get storage id, throws exception if function node is not resolved
|
||||
const StorageID & getStorageID() const;
|
||||
@ -106,6 +106,11 @@ public:
|
||||
/// Get storage snapshot, throws exception if function node is not resolved
|
||||
const StorageSnapshotPtr & getStorageSnapshot() const;
|
||||
|
||||
const std::vector<size_t> & getUnresolvedArgumentIndexes() const
|
||||
{
|
||||
return unresolved_arguments_indexes;
|
||||
}
|
||||
|
||||
/// Return true if table function node has table expression modifiers, false otherwise
|
||||
bool hasTableExpressionModifiers() const
|
||||
{
|
||||
@ -164,6 +169,7 @@ private:
|
||||
StoragePtr storage;
|
||||
StorageID storage_id;
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
std::vector<size_t> unresolved_arguments_indexes;
|
||||
std::optional<TableExpressionModifiers> table_expression_modifiers;
|
||||
SettingsChanges settings_changes;
|
||||
|
||||
|
@ -16,11 +16,6 @@ namespace fs = std::filesystem;
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
CachedObjectStorage::CachedObjectStorage(
|
||||
ObjectStoragePtr object_storage_,
|
||||
FileCachePtr cache_,
|
||||
@ -79,8 +74,6 @@ std::unique_ptr<ReadBufferFromFileBase> CachedObjectStorage::readObjects( /// NO
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const
|
||||
{
|
||||
if (objects.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Received empty list of objects to read");
|
||||
return object_storage->readObjects(objects, patchSettings(read_settings), read_hint, file_size);
|
||||
}
|
||||
|
||||
|
@ -3311,7 +3311,7 @@ bool StorageReplicatedMergeTree::scheduleDataProcessingJob(BackgroundJobsAssigne
|
||||
auto job_type = selected_entry->log_entry->type;
|
||||
|
||||
/// Depending on entry type execute in fetches (small) pool or big merge_mutate pool
|
||||
if (job_type == LogEntry::GET_PART)
|
||||
if (job_type == LogEntry::GET_PART || job_type == LogEntry::ATTACH_PART)
|
||||
{
|
||||
assignee.scheduleFetchTask(std::make_shared<ExecutableLambdaAdapter>(
|
||||
[this, selected_entry] () mutable
|
||||
|
@ -130,7 +130,7 @@ public:
|
||||
return true;
|
||||
}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
auto * function_node = node->as<FunctionNode>();
|
||||
auto * join_node = node->as<JoinNode>();
|
||||
|
@ -129,7 +129,7 @@ position с конца строки.
|
||||
|
||||
Атомарно удаляет таблицу перед созданием новой, если такая была.
|
||||
|
||||
## * Приведение типов для IN (subquery).
|
||||
## + Приведение типов для IN (subquery).
|
||||
|
||||
`SELECT 1 IN (SELECT -1 UNION ALL SELECT 1)`
|
||||
|
||||
@ -205,12 +205,12 @@ https://clickhouse.com/docs/en/operations/table_engines/external_data/
|
||||
|
||||
## Возможность задавать параметры соединений для табличных функций, движков таблиц и для реплик из отдельных разделов конфигурации.
|
||||
|
||||
## Настройка rollup_use_nulls.
|
||||
## + Настройка rollup_use_nulls.
|
||||
|
||||
Upd: it is named "group_by_use_nulls".
|
||||
|
||||
## + Настройка cast_keep_nullable.
|
||||
|
||||
## Функция bitEquals для сравнения произвольных типов данных побитово.
|
||||
## Функция bitEquals для сравнения произвольных типов данных побитово
|
||||
|
||||
## Функция serialize для implementation specific non portable non backwards compatible сериализации любого типа данных в набор байт.
|
||||
|
||||
## Функция bitEquals и оператор <=>.
|
||||
|
@ -1,3 +1,6 @@
|
||||
1 []
|
||||
1 []
|
||||
[]
|
||||
1 []
|
||||
1 []
|
||||
[]
|
||||
|
@ -5,11 +5,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
for DISK in s3_disk s3_cache
|
||||
do
|
||||
${CLICKHOUSE_CLIENT} -n --query "
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (id Int32, empty Array(Int32))
|
||||
ENGINE=MergeTree ORDER BY id
|
||||
SETTINGS min_rows_for_wide_part=0, min_bytes_for_wide_part=0, disk='s3_disk';
|
||||
SETTINGS min_rows_for_wide_part=0, min_bytes_for_wide_part=0, disk='$DISK';
|
||||
|
||||
INSERT INTO test (id) VALUES (1);
|
||||
SELECT * FROM test;
|
||||
@ -25,3 +27,4 @@ ${CLICKHOUSE_CLIENT} -n --query "
|
||||
SELECT * FROM test;
|
||||
SELECT empty FROM test;
|
||||
"
|
||||
done
|
||||
|
Loading…
Reference in New Issue
Block a user